Skip to content

Commit 05974f9

Browse files
gengliangwangcloud-fan
authored andcommitted
[SPARK-25133][SQL][DOC] Avro data source guide
## What changes were proposed in this pull request? Create documentation for AVRO data source. The new page will be linked in https://spark.apache.org/docs/latest/sql-programming-guide.html For preview please unzip the following file: [AvroDoc.zip](https://github.com/apache/spark/files/2313011/AvroDoc.zip) Closes #22121 from gengliangwang/avroDoc. Authored-by: Gengliang Wang <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 1747469 commit 05974f9

File tree

2 files changed

+383
-0
lines changed

2 files changed

+383
-0
lines changed

docs/avro-data-source-guide.md

Lines changed: 380 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,380 @@
1+
---
2+
layout: global
3+
title: Apache Avro Data Source Guide
4+
---
5+
6+
* This will become a table of contents (this text will be scraped).
7+
{:toc}
8+
9+
Since Spark 2.4 release, [Spark SQL](https://spark.apache.org/docs/latest/sql-programming-guide.html) provides built-in support for reading and writing Apache Avro data.
10+
11+
## Deploying
12+
The `spark-avro` module is external and not included in `spark-submit` or `spark-shell` by default.
13+
14+
As with any Spark applications, `spark-submit` is used to launch your application. `spark-avro_{{site.SCALA_BINARY_VERSION}}`
15+
and its dependencies can be directly added to `spark-submit` using `--packages`, such as,
16+
17+
./bin/spark-submit --packages org.apache.spark:spark-avro_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
18+
19+
For experimenting on `spark-shell`, you can also use `--packages` to add `org.apache.spark:spark-avro_{{site.SCALA_BINARY_VERSION}}` and its dependencies directly,
20+
21+
./bin/spark-shell --packages org.apache.spark:spark-avro_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
22+
23+
See [Application Submission Guide](submitting-applications.html) for more details about submitting applications with external dependencies.
24+
25+
## Load and Save Functions
26+
27+
Since `spark-avro` module is external, there is no `.avro` API in
28+
`DataFrameReader` or `DataFrameWriter`.
29+
30+
To load/save data in Avro format, you need to specify the data source option `format` as `avro`(or `org.apache.spark.sql.avro`).
31+
<div class="codetabs">
32+
<div data-lang="scala" markdown="1">
33+
{% highlight scala %}
34+
35+
val usersDF = spark.read.format("avro").load("examples/src/main/resources/users.avro")
36+
usersDF.select("name", "favorite_color").write.format("avro").save("namesAndFavColors.avro")
37+
38+
{% endhighlight %}
39+
</div>
40+
<div data-lang="java" markdown="1">
41+
{% highlight java %}
42+
43+
Dataset<Row> usersDF = spark.read().format("avro").load("examples/src/main/resources/users.avro");
44+
usersDF.select("name", "favorite_color").write().format("avro").save("namesAndFavColors.avro");
45+
46+
{% endhighlight %}
47+
</div>
48+
<div data-lang="python" markdown="1">
49+
{% highlight python %}
50+
51+
df = spark.read.format("avro").load("examples/src/main/resources/users.avro")
52+
df.select("name", "favorite_color").write.format("avro").save("namesAndFavColors.avro")
53+
54+
{% endhighlight %}
55+
</div>
56+
<div data-lang="r" markdown="1">
57+
{% highlight r %}
58+
59+
df <- read.df("examples/src/main/resources/users.avro", "avro")
60+
write.df(select(df, "name", "favorite_color"), "namesAndFavColors.avro", "avro")
61+
62+
{% endhighlight %}
63+
</div>
64+
</div>
65+
66+
## to_avro() and from_avro()
67+
The Avro package provides function `to_avro` to encode a column as binary in Avro
68+
format, and `from_avro()` to decode Avro binary data into a column. Both functions transform one column to
69+
another column, and the input/output SQL data type can be complex type or primitive type.
70+
71+
Using Avro record as columns are useful when reading from or writing to a streaming source like Kafka. Each
72+
Kafka key-value record will be augmented with some metadata, such as the ingestion timestamp into Kafka, the offset in Kafka, etc.
73+
* If the "value" field that contains your data is in Avro, you could use `from_avro()` to extract your data, enrich it, clean it, and then push it downstream to Kafka again or write it out to a file.
74+
* `to_avro()` can be used to turn structs into Avro records. This method is particularly useful when you would like to re-encode multiple columns into a single one when writing data out to Kafka.
75+
76+
Both functions are currently only available in Scala and Java.
77+
78+
<div class="codetabs">
79+
<div data-lang="scala" markdown="1">
80+
{% highlight scala %}
81+
import org.apache.spark.sql.avro._
82+
83+
// `from_avro` requires Avro schema in JSON string format.
84+
val jsonFormatSchema = new String(Files.readAllBytes(Paths.get("./examples/src/main/resources/user.avsc")))
85+
86+
val df = spark
87+
.readStream
88+
.format("kafka")
89+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
90+
.option("subscribe", "topic1")
91+
.load()
92+
93+
// 1. Decode the Avro data into a struct;
94+
// 2. Filter by column `favorite_color`;
95+
// 3. Encode the column `name` in Avro format.
96+
val output = df
97+
.select(from_avro('value, jsonFormatSchema) as 'user)
98+
.where("user.favorite_color == \"red\"")
99+
.select(to_avro($"user.name") as 'value)
100+
101+
val query = output
102+
.writeStream
103+
.format("kafka")
104+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
105+
.option("topic", "topic2")
106+
.start()
107+
108+
{% endhighlight %}
109+
</div>
110+
<div data-lang="java" markdown="1">
111+
{% highlight java %}
112+
import org.apache.spark.sql.avro.*;
113+
114+
// `from_avro` requires Avro schema in JSON string format.
115+
String jsonFormatSchema = new String(Files.readAllBytes(Paths.get("./examples/src/main/resources/user.avsc")));
116+
117+
Dataset<Row> df = spark
118+
.readStream()
119+
.format("kafka")
120+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
121+
.option("subscribe", "topic1")
122+
.load();
123+
124+
// 1. Decode the Avro data into a struct;
125+
// 2. Filter by column `favorite_color`;
126+
// 3. Encode the column `name` in Avro format.
127+
Dataset<Row> output = df
128+
.select(from_avro(col("value"), jsonFormatSchema).as("user"))
129+
.where("user.favorite_color == \"red\"")
130+
.select(to_avro(col("user.name")).as("value"));
131+
132+
StreamingQuery query = output
133+
.writeStream()
134+
.format("kafka")
135+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
136+
.option("topic", "topic2")
137+
.start();
138+
139+
{% endhighlight %}
140+
</div>
141+
</div>
142+
143+
## Data Source Option
144+
145+
Data source options of Avro can be set using the `.option` method on `DataFrameReader` or `DataFrameWriter`.
146+
<table class="table">
147+
<tr><th><b>Property Name</b></th><th><b>Default</b></th><th><b>Meaning</b></th><th><b>Scope</b></th></tr>
148+
<tr>
149+
<td><code>avroSchema</code></td>
150+
<td>None</td>
151+
<td>Optional Avro schema provided by an user in JSON format. The date type and naming of record fields
152+
should match the input Avro data or Catalyst data, otherwise the read/write action will fail.</td>
153+
<td>read and write</td>
154+
</tr>
155+
<tr>
156+
<td><code>recordName</code></td>
157+
<td>topLevelRecord</td>
158+
<td>Top level record name in write result, which is required in Avro spec.</td>
159+
<td>write</td>
160+
</tr>
161+
<tr>
162+
<td><code>recordNamespace</code></td>
163+
<td>""</td>
164+
<td>Record namespace in write result.</td>
165+
<td>write</td>
166+
</tr>
167+
<tr>
168+
<td><code>ignoreExtension</code></td>
169+
<td>true</td>
170+
<td>The option controls ignoring of files without <code>.avro</code> extensions in read.<br> If the option is enabled, all files (with and without <code>.avro</code> extension) are loaded.</td>
171+
<td>read</td>
172+
</tr>
173+
<tr>
174+
<td><code>compression</code></td>
175+
<td>snappy</td>
176+
<td>The <code>compression</code> option allows to specify a compression codec used in write.<br>
177+
Currently supported codecs are <code>uncompressed</code>, <code>snappy</code>, <code>deflate</code>, <code>bzip2</code> and <code>xz</code>.<br> If the option is not set, the configuration <code>spark.sql.avro.compression.codec</code> config is taken into account.</td>
178+
<td>write</td>
179+
</tr>
180+
</table>
181+
182+
## Configuration
183+
Configuration of Avro can be done using the `setConf` method on SparkSession or by running `SET key=value` commands using SQL.
184+
<table class="table">
185+
<tr><th><b>Property Name</b></th><th><b>Default</b></th><th><b>Meaning</b></th></tr>
186+
<tr>
187+
<td>spark.sql.legacy.replaceDatabricksSparkAvro.enabled</td>
188+
<td>true</td>
189+
<td>If it is set to true, the data source provider <code>com.databricks.spark.avro</code> is mapped to the built-in but external Avro data source module for backward compatibility.</td>
190+
</tr>
191+
<tr>
192+
<td>spark.sql.avro.compression.codec</td>
193+
<td>snappy</td>
194+
<td>Compression codec used in writing of AVRO files. Supported codecs: uncompressed, deflate, snappy, bzip2 and xz. Default codec is snappy.</td>
195+
</tr>
196+
<tr>
197+
<td>spark.sql.avro.deflate.level</td>
198+
<td>-1</td>
199+
<td>Compression level for the deflate codec used in writing of AVRO files. Valid value must be in the range of from 1 to 9 inclusive or -1. The default value is -1 which corresponds to 6 level in the current implementation.</td>
200+
</tr>
201+
</table>
202+
203+
## Compatibility with Databricks spark-avro
204+
This Avro data source module is originally from and compatible with Databricks's open source repository
205+
[spark-avro](https://github.com/databricks/spark-avro).
206+
207+
By default with the SQL configuration `spark.sql.legacy.replaceDatabricksSparkAvro.enabled` enabled, the data source provider `com.databricks.spark.avro` is
208+
mapped to this built-in Avro module. For the Spark tables created with `Provider` property as `com.databricks.spark.avro` in
209+
catalog meta store, the mapping is essential to load these tables if you are using this built-in Avro module.
210+
211+
Note in Databricks's [spark-avro](https://github.com/databricks/spark-avro), implicit classes
212+
`AvroDataFrameWriter` and `AvroDataFrameReader` were created for shortcut function `.avro()`. In this
213+
built-in but external module, both implicit classes are removed. Please use `.format("avro")` in
214+
`DataFrameWriter` or `DataFrameReader` instead, which should be clean and good enough.
215+
216+
If you prefer using your own build of `spark-avro` jar file, you can simply disable the configuration
217+
`spark.sql.legacy.replaceDatabricksSparkAvro.enabled`, and use the option `--jars` on deploying your
218+
applications. Read the [Advanced Dependency Management](https://spark.apache
219+
.org/docs/latest/submitting-applications.html#advanced-dependency-management) section in Application
220+
Submission Guide for more details.
221+
222+
## Supported types for Avro -> Spark SQL conversion
223+
Currently Spark supports reading all [primitive types](https://avro.apache.org/docs/1.8.2/spec.html#schema_primitive) and [complex types](https://avro.apache.org/docs/1.8.2/spec.html#schema_complex) under records of Avro.
224+
<table class="table">
225+
<tr><th><b>Avro type</b></th><th><b>Spark SQL type</b></th></tr>
226+
<tr>
227+
<td>boolean</td>
228+
<td>BooleanType</td>
229+
</tr>
230+
<tr>
231+
<td>int</td>
232+
<td>IntegerType</td>
233+
</tr>
234+
<tr>
235+
<td>long</td>
236+
<td>LongType</td>
237+
</tr>
238+
<tr>
239+
<td>float</td>
240+
<td>FloatType</td>
241+
</tr>
242+
<tr>
243+
<td>double</td>
244+
<td>DoubleType</td>
245+
</tr>
246+
<tr>
247+
<td>string</td>
248+
<td>StringType</td>
249+
</tr>
250+
<tr>
251+
<td>enum</td>
252+
<td>StringType</td>
253+
</tr>
254+
<tr>
255+
<td>fixed</td>
256+
<td>BinaryType</td>
257+
</tr>
258+
<tr>
259+
<td>bytes</td>
260+
<td>BinaryType</td>
261+
</tr>
262+
<tr>
263+
<td>record</td>
264+
<td>StructType</td>
265+
</tr>
266+
<tr>
267+
<td>array</td>
268+
<td>ArrayType</td>
269+
</tr>
270+
<tr>
271+
<td>map</td>
272+
<td>MapType</td>
273+
</tr>
274+
<tr>
275+
<td>union</td>
276+
<td>See below</td>
277+
</tr>
278+
</table>
279+
280+
In addition to the types listed above, it supports reading `union` types. The following three types are considered basic `union` types:
281+
282+
1. `union(int, long)` will be mapped to LongType.
283+
2. `union(float, double)` will be mapped to DoubleType.
284+
3. `union(something, null)`, where something is any supported Avro type. This will be mapped to the same Spark SQL type as that of something, with nullable set to true.
285+
All other union types are considered complex. They will be mapped to StructType where field names are member0, member1, etc., in accordance with members of the union. This is consistent with the behavior when converting between Avro and Parquet.
286+
287+
It also supports reading the following Avro [logical types](https://avro.apache.org/docs/1.8.2/spec.html#Logical+Types):
288+
289+
<table class="table">
290+
<tr><th><b>Avro logical type</b></th><th><b>Avro type</b></th><th><b>Spark SQL type</b></th></tr>
291+
<tr>
292+
<td>date</td>
293+
<td>int</td>
294+
<td>DateType</td>
295+
</tr>
296+
<tr>
297+
<td>timestamp-millis</td>
298+
<td>long</td>
299+
<td>TimestampType</td>
300+
</tr>
301+
<tr>
302+
<td>timestamp-micros</td>
303+
<td>long</td>
304+
<td>TimestampType</td>
305+
</tr>
306+
<tr>
307+
<td>decimal</td>
308+
<td>fixed</td>
309+
<td>DecimalType</td>
310+
</tr>
311+
<tr>
312+
<td>decimal</td>
313+
<td>bytes</td>
314+
<td>DecimalType</td>
315+
</tr>
316+
</table>
317+
At the moment, it ignores docs, aliases and other properties present in the Avro file.
318+
319+
## Supported types for Spark SQL -> Avro conversion
320+
Spark supports writing of all Spark SQL types into Avro. For most types, the mapping from Spark types to Avro types is straightforward (e.g. IntegerType gets converted to int); however, there are a few special cases which are listed below:
321+
322+
<table class="table">
323+
<tr><th><b>Spark SQL type</b></th><th><b>Avro type</b></th><th><b>Avro logical type</b></th></tr>
324+
<tr>
325+
<td>ByteType</td>
326+
<td>int</td>
327+
<td></td>
328+
</tr>
329+
<tr>
330+
<td>ShortType</td>
331+
<td>int</td>
332+
<td></td>
333+
</tr>
334+
<tr>
335+
<td>BinaryType</td>
336+
<td>bytes</td>
337+
<td></td>
338+
</tr>
339+
<tr>
340+
<td>DateType</td>
341+
<td>int</td>
342+
<td>date</td>
343+
</tr>
344+
<tr>
345+
<td>TimestampType</td>
346+
<td>long</td>
347+
<td>timestamp-micros</td>
348+
</tr>
349+
<tr>
350+
<td>DecimalType</td>
351+
<td>fixed</td>
352+
<td>decimal</td>
353+
</tr>
354+
</table>
355+
356+
You can also specify the whole output Avro schema with the option `avroSchema`, so that Spark SQL types can be converted into other Avro types. The following conversions are not applied by default and require user specified Avro schema:
357+
358+
<table class="table">
359+
<tr><th><b>Spark SQL type</b></th><th><b>Avro type</b></th><th><b>Avro logical type</b></th></tr>
360+
<tr>
361+
<td>BinaryType</td>
362+
<td>fixed</td>
363+
<td></td>
364+
</tr>
365+
<tr>
366+
<td>StringType</td>
367+
<td>enum</td>
368+
<td></td>
369+
</tr>
370+
<tr>
371+
<td>TimestampType</td>
372+
<td>long</td>
373+
<td>timestamp-millis</td>
374+
</tr>
375+
<tr>
376+
<td>DecimalType</td>
377+
<td>bytes</td>
378+
<td>decimal</td>
379+
</tr>
380+
</table>

docs/sql-programming-guide.md

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1482,6 +1482,9 @@ SELECT * FROM resultTable
14821482
</div>
14831483
</div>
14841484

1485+
## Avro Files
1486+
See the [Apache Avro Data Source Guide](avro-data-source-guide.html).
1487+
14851488
## Troubleshooting
14861489

14871490
* The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs.

0 commit comments

Comments
 (0)