## What changes were proposed in this pull request?
This builds on top of SPARK-24748 to report 'offset lag' as a custom metrics for Kafka structured streaming source.
This lag is the difference between the latest offsets in Kafka the time the metrics is reported (just after a micro-batch completes) and the latest offset Spark has processed. It can be 0 (or close to 0) if spark keeps up with the rate at which messages are ingested into Kafka topics in steady state. This measures how far behind the spark source has fallen behind (per partition) and can aid in tuning the application.
## How was this patch tested?
Existing and new unit tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#21819 from arunmahadevan/SPARK-24863.
Authored-by: Arun Mahadevan <arunm@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
KafkaContinuousSinkSuite leaks a Kafka cluster because both KafkaSourceTest and KafkaContinuousSinkSuite create a Kafka cluster but `afterAll` only shuts down one cluster. This leaks a Kafka cluster and causes that some Kafka thread crash and kill JVM when SBT is trying to clean up tests.
This PR fixes the leak and also adds a shut down hook to detect Kafka cluster leak.
In additions, it also fixes `AdminClient` leak and cleans up cached producers (When a record is writtn using a producer, the producer will keep refreshing the topic and I don't find an API to clear it except closing the producer) to eliminate the following annoying logs:
```
8/13 15:34:42.568 kafka-admin-client-thread | adminclient-4 WARN NetworkClient: [AdminClient clientId=adminclient-4] Connection to node 0 could not be established. Broker may not be available.
18/08/13 15:34:42.570 kafka-admin-client-thread | adminclient-6 WARN NetworkClient: [AdminClient clientId=adminclient-6] Connection to node 0 could not be established. Broker may not be available.
18/08/13 15:34:42.606 kafka-admin-client-thread | adminclient-8 WARN NetworkClient: [AdminClient clientId=adminclient-8] Connection to node -1 could not be established. Broker may not be available.
18/08/13 15:34:42.729 kafka-producer-network-thread | producer-797 WARN NetworkClient: [Producer clientId=producer-797] Connection to node -1 could not be established. Broker may not be available.
18/08/13 15:34:42.906 kafka-producer-network-thread | producer-1598 WARN NetworkClient: [Producer clientId=producer-1598] Connection to node 0 could not be established. Broker may not be available.
```
I also reverted b5eb54244e introduced by #22097 since it doesn't help.
## How was this patch tested?
Jenkins
Closes#22106 from zsxwing/SPARK-25116.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
Remove jets3t dependency, and bouncy castle which it brings in; update licenses and deps
Note this just takes over https://github.com/apache/spark/pull/21146
## How was this patch tested?
Existing tests.
Closes#22081 from srowen/SPARK-23654.
Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
I'm still seeing the Kafka tests failed randomly due to `kafka.zookeeper.ZooKeeperClientTimeoutException: Timed out waiting for connection while in state: CONNECTING`. I checked the test output and saw zookeeper picked up an ipv6 address. Most details can be found in https://issues.apache.org/jira/browse/KAFKA-7193
This PR just uses `127.0.0.1` rather than `localhost` to make sure zookeeper will never use an ipv6 address.
## How was this patch tested?
Jenkins
Closes#22097 from zsxwing/fix-zookeeper-connect.
Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request?
With code changes in https://github.com/apache/spark/pull/21847 , Spark can write out to Avro file as per user provided output schema.
To make it more robust and user friendly, we should validate the Avro schema before tasks launched.
Also we should support output logical decimal type as BYTES (By default we output as FIXED)
## How was this patch tested?
Unit test
Closes#22094 from gengliangwang/AvroSerializerMatch.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
In PR https://github.com/apache/spark/pull/21984 and https://github.com/apache/spark/pull/21935 , the related test cases are using binary files created by Python scripts.
Generate the binary files in test suite to make it more transparent. Also we can
Also move the related test cases to a new file `AvroLogicalTypeSuite.scala`.
## How was this patch tested?
Unit test.
Closes#22091 from gengliangwang/logicalType_suite.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Support Avro logical date type:
https://avro.apache.org/docs/1.8.2/spec.html#Decimal
## How was this patch tested?
Unit test
Closes#22037 from gengliangwang/avro_decimal.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Allows `avroSchema` option to be specified on write, allowing a user to specify a schema in cases where this is required. A trivial use case is reading in an avro dataset, making some small adjustment to a column or columns and writing out using the same schema. Implicit schema creation from SQL Struct results in a schema that while for the most part, is functionally similar, is not necessarily compatible.
Allows `fixed` Field type to be utilized for records of specified `avroSchema`
## How was this patch tested?
Unit tests in AvroSuite are extended to test this with enum and fixed types.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#21847 from lindblombr/specify_schema_on_write.
Lead-authored-by: Brian Lindblom <blindblom@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
## What changes were proposed in this pull request?
This PR fixes typo regarding `auxiliary verb + verb[s]`. This is a follow-on of #21956.
## How was this patch tested?
N/A
Closes#22040 from kiszk/spellcheck1.
Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Support Avro logical date type:
https://avro.apache.org/docs/1.8.2/spec.html#Date
## How was this patch tested?
Unit test
Closes#21984 from gengliangwang/avro_date.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
A follow up of #21118
Since we use `InternalRow` in the read API of data source v2, we should do the same thing for the write API.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#21948 from cloud-fan/row-write.
## What changes were proposed in this pull request?
This small fix adds a `consumer.release()` call to `KafkaSourceRDD` in the case where we've retrieved offsets from Kafka, but the `fromOffset` is equal to the `lastOffset`, meaning there is no new data to read for a particular topic partition. Up until now, we'd just return an empty iterator without closing the consumer which would cause a FD leak.
If accepted, this pull request should be merged into master as well.
## How was this patch tested?
Haven't ran any specific tests, would love help on how to test methods running inside `RDD.compute`.
Author: Yuval Itzchakov <yuval.itzchakov@clicktale.com>
Closes#21997 from YuvalItzchakov/master.
## What changes were proposed in this pull request?
Increase ZK timeout and harmonize configs across Kafka tests to resol…ve potentially flaky test failure
## How was this patch tested?
Existing tests
Author: Sean Owen <srowen@gmail.com>
Closes#21995 from srowen/SPARK-18057.3.
## What changes were proposed in this pull request?
Update to kafka 2.0.0 in streaming-kafka module, and remove override for Scala 2.12. It won't compile for 2.12 otherwise.
## How was this patch tested?
Existing tests.
Author: Sean Owen <srowen@gmail.com>
Closes#21955 from srowen/SPARK-18057.2.
## What changes were proposed in this pull request?
When lindblombr at apple developed [SPARK-24855](https://github.com/apache/spark/pull/21847) to support specified schema on write, we found a performance regression in Avro writer for our dataset.
With this PR, the performance is improved, but not as good as Spark 2.3 + the old avro writer. There must be something we miss which we need to investigate further.
Spark 2.4
```
spark git:(master) ./build/mvn -DskipTests clean package
spark git:(master) bin/spark-shell --jars external/avro/target/spark-avro_2.11-2.4.0-SNAPSHOT.jar
```
Spark 2.3 + databricks avro
```
spark git:(branch-2.3) ./build/mvn -DskipTests clean package
spark git:(branch-2.3) bin/spark-shell --packages com.databricks:spark-avro_2.11:4.0.0
```
Current master:
```
+-------+--------------------+
|summary| writeTimes|
+-------+--------------------+
| count| 100|
| mean| 2.95621|
| stddev|0.030895815479469294|
| min| 2.915|
| max| 3.049|
+-------+--------------------+
+-------+--------------------+
|summary| readTimes|
+-------+--------------------+
| count| 100|
| mean| 0.31072999999999995|
| stddev|0.054139709842390006|
| min| 0.259|
| max| 0.692|
+-------+--------------------+
```
Current master with this PR:
```
+-------+--------------------+
|summary| writeTimes|
+-------+--------------------+
| count| 100|
| mean| 2.5804300000000002|
| stddev|0.011175600225672079|
| min| 2.558|
| max| 2.62|
+-------+--------------------+
+-------+--------------------+
|summary| readTimes|
+-------+--------------------+
| count| 100|
| mean| 0.29922000000000004|
| stddev|0.058261961532514166|
| min| 0.251|
| max| 0.732|
+-------+--------------------+
```
Spark 2.3 + databricks avro:
```
+-------+--------------------+
|summary| writeTimes|
+-------+--------------------+
| count| 100|
| mean| 1.7730500000000005|
| stddev|0.025199156230863575|
| min| 1.729|
| max| 1.833|
+-------+--------------------+
+-------+-------------------+
|summary| readTimes|
+-------+-------------------+
| count| 100|
| mean| 0.29715|
| stddev|0.05685643358850465|
| min| 0.258|
| max| 0.718|
+-------+-------------------+
```
The following is the test code to reproduce the result.
```scala
spark.sqlContext.setConf("spark.sql.avro.compression.codec", "uncompressed")
val sparkSession = spark
import sparkSession.implicits._
val df = spark.sparkContext.range(1, 3000).repartition(1).map { uid =>
val features = Array.fill(16000)(scala.math.random)
(uid, scala.math.random, java.util.UUID.randomUUID().toString, java.util.UUID.randomUUID().toString, features)
}.toDF("uid", "random", "uuid1", "uuid2", "features").cache()
val size = df.count()
// Write into ramdisk to rule out the disk IO impact
val tempSaveDir = s"/Volumes/ramdisk/${java.util.UUID.randomUUID()}/"
val n = 150
val writeTimes = new Array[Double](n)
var i = 0
while (i < n) {
val t1 = System.currentTimeMillis()
df.write
.format("com.databricks.spark.avro")
.mode("overwrite")
.save(tempSaveDir)
val t2 = System.currentTimeMillis()
writeTimes(i) = (t2 - t1) / 1000.0
i += 1
}
df.unpersist()
// The first 50 runs are for warm-up
val readTimes = new Array[Double](n)
i = 0
while (i < n) {
val t1 = System.currentTimeMillis()
val readDF = spark.read.format("com.databricks.spark.avro").load(tempSaveDir)
assert(readDF.count() == size)
val t2 = System.currentTimeMillis()
readTimes(i) = (t2 - t1) / 1000.0
i += 1
}
spark.sparkContext.parallelize(writeTimes.slice(50, 150)).toDF("writeTimes").describe("writeTimes").show()
spark.sparkContext.parallelize(readTimes.slice(50, 150)).toDF("readTimes").describe("readTimes").show()
```
## How was this patch tested?
Existing tests.
Author: DB Tsai <d_tsai@apple.com>
Author: Brian Lindblom <blindblom@apple.com>
Closes#21952 from dbtsai/avro-performance-fix.
## What changes were proposed in this pull request?
Currently the output namespace is starting with ".", e.g. `.topLevelRecord`
Although it is valid according to Avro spec, we should remove the starting dot in case of failures when the output Avro file is read by other lib:
https://github.com/linkedin/goavro/pull/96
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21974 from gengliangwang/avro_namespace.
## What changes were proposed in this pull request?
Support reading/writing Avro logical timestamp type with different precisions
https://avro.apache.org/docs/1.8.2/spec.html#Timestamp+%28millisecond+precision%29
To specify the output timestamp type, use Dataframe option `outputTimestampType` or SQL config `spark.sql.avro.outputTimestampType`. The supported values are
* `TIMESTAMP_MICROS`
* `TIMESTAMP_MILLIS`
The default output type is `TIMESTAMP_MICROS`
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21935 from gengliangwang/avro_timestamp.
## What changes were proposed in this pull request?
This PR addresses issues 2,3 in this [document](https://docs.google.com/document/d/1fbkjEL878witxVQpOCbjlvOvadHtVjYXeB-2mgzDTvk).
* We modified the closure cleaner to identify closures that are implemented via the LambdaMetaFactory mechanism (serializedLambdas) (issue2).
* We also fix the issue due to scala/bug#11016. There are two options for solving the Unit issue, either add () at the end of the closure or use the trick described in the doc. Otherwise overloading resolution does not work (we are not going to eliminate either of the methods) here. Compiler tries to adapt to Unit and makes these two methods candidates for overloading, when there is polymorphic overloading there is no ambiguity (that is the workaround implemented). This does not look that good but it serves its purpose as we need to support two different uses for method: `addTaskCompletionListener`. One that passes a TaskCompletionListener and one that passes a closure that is wrapped with a TaskCompletionListener later on (issue3).
Note: regarding issue 1 in the doc the plan is:
> Do Nothing. Don’t try to fix this as this is only a problem for Java users who would want to use 2.11 binaries. In that case they can cast to MapFunction to be able to utilize lambdas. In Spark 3.0.0 the API should be simplified so that this issue is removed.
## How was this patch tested?
This was manually tested:
```./dev/change-scala-version.sh 2.12
./build/mvn -DskipTests -Pscala-2.12 clean package
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.serializer.ProactiveClosureSerializationSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.util.ClosureCleanerSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.streaming.DStreamClosureSuite -Dtest=None```
Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Closes#21930 from skonto/scala2.12-sup.
## What changes were proposed in this pull request?
This PR upgrades to the Kafka 2.0.0 release where KIP-266 is integrated.
## How was this patch tested?
This PR uses existing Kafka related unit tests
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: tedyu <yuzhihong@gmail.com>
Closes#21488 from tedyu/master.
## What changes were proposed in this pull request?
In the PR, I propose to support `LZMA2` (`XZ`) and `BZIP2` compressions by `AVRO` datasource in write since the codecs may have better characteristics like compression ratio and speed comparing to already supported `snappy` and `deflate` codecs.
## How was this patch tested?
It was tested manually and by an existing test which was extended to check the `xz` and `bzip2` compressions.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes#21902 from MaxGekk/avro-xz-bzip2.
## What changes were proposed in this pull request?
This pr supported Date/Timestamp in a JDBC partition column (a numeric column is only supported in the master). This pr also modified code to verify a partition column type;
```
val jdbcTable = spark.read
.option("partitionColumn", "text")
.option("lowerBound", "aaa")
.option("upperBound", "zzz")
.option("numPartitions", 2)
.jdbc("jdbc:postgresql:postgres", "t", options)
// with this pr
org.apache.spark.sql.AnalysisException: Partition column type should be numeric, date, or timestamp, but string found.;
at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.verifyAndGetNormalizedPartitionColumn(JDBCRelation.scala:165)
at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.columnPartition(JDBCRelation.scala:85)
at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:36)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:317)
// without this pr
java.lang.NumberFormatException: For input string: "aaa"
at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
at java.lang.Long.parseLong(Long.java:589)
at java.lang.Long.parseLong(Long.java:631)
at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:277)
```
Closes#19999
## How was this patch tested?
Added tests in `JDBCSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#21834 from maropu/SPARK-22814.
## What changes were proposed in this pull request?
Looks Avro uses direct `getLogger` to create a SLF4J logger. Should better use `internal.Logging` instead.
## How was this patch tested?
Exiting tests.
Author: hyukjinkwon <gurwls223@apache.org>
Closes#21914 from HyukjinKwon/avro-log.
## What changes were proposed in this pull request?
Add one more test case for `com.databricks.spark.avro`.
## How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#21906 from gatorsmile/avro.
## What changes were proposed in this pull request?
In the PR, I added new option for Avro datasource - `compression`. The option allows to specify compression codec for saved Avro files. This option is similar to `compression` option in another datasources like `JSON` and `CSV`.
Also I added the SQL configs `spark.sql.avro.compression.codec` and `spark.sql.avro.deflate.level`. I put the configs into `SQLConf`. If the `compression` option is not specified by an user, the first SQL config is taken into account.
## How was this patch tested?
I added new test which read meta info from written avro files and checks `avro.codec` property.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes#21837 from MaxGekk/avro-compression.
## What changes were proposed in this pull request?
In most cases, we should use `spark.sessionState.newHadoopConf()` instead of `sparkContext.hadoopConfiguration`, so that the hadoop configurations specified in Spark session
configuration will come into effect.
Add a rule matching `spark.sparkContext.hadoopConfiguration` or `spark.sqlContext.sparkContext.hadoopConfiguration` to prevent the usage.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21873 from gengliangwang/linterRule.
## What changes were proposed in this pull request?
This PR aims to the followings.
1. Like `com.databricks.spark.csv` mapping, we had better map `com.databricks.spark.avro` to built-in Avro data source.
2. Remove incorrect error message, `Please find an Avro package at ...`.
## How was this patch tested?
Pass the newly added tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#21878 from dongjoon-hyun/SPARK-24924.
## What changes were proposed in this pull request?
After rethinking on the artifactId, I think it should be `spark-avro` instead of `spark-sql-avro`, which is simpler, and consistent with the previous artifactId. I think we need to change it before Spark 2.4 release.
Also a tiny change: use `spark.sessionState.newHadoopConf()` to get the hadoop configuration, thus the related hadoop configurations in SQLConf will come into effect.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21866 from gengliangwang/avro_followup.
## What changes were proposed in this pull request?
This updates the DataSourceV2 API to use InternalRow instead of Row for the default case with no scan mix-ins.
Support for readers that produce Row is added through SupportsDeprecatedScanRow, which matches the previous API. Readers that used Row now implement this class and should be migrated to InternalRow.
Readers that previously implemented SupportsScanUnsafeRow have been migrated to use no SupportsScan mix-ins and produce InternalRow.
## How was this patch tested?
This uses existing tests.
Author: Ryan Blue <blue@apache.org>
Closes#21118 from rdblue/SPARK-23325-datasource-v2-internal-row.
## What changes were proposed in this pull request?
To implement the method `buildReader` in `FileFormat`, it is required to serialize the hadoop configuration for executors.
Previous spark-avro uses its own class `SerializableConfiguration` for the serialization. As now it is part of Spark, we can use SerializableConfiguration in Spark util to deduplicate the code.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21846 from gengliangwang/removeSerializableConfiguration.
## What changes were proposed in this pull request?
As per Reynold's comment: https://github.com/apache/spark/pull/21742#discussion_r203496489
It makes sense to remove the implicit class AvroDataFrameWriter/AvroDataFrameReader, since the Avro package is external module.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21841 from gengliangwang/removeImplicit.
## What changes were proposed in this pull request?
1. Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value.
2. Add a new function to_avro for converting a column into binary of avro format with the specified schema.
I created #21774 for this, but it failed the build https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7902/
Additional changes In this PR:
1. Add `scalacheck` dependency in pom.xml to resolve the failure.
2. Update the `log4j.properties` to make it consistent with other modules.
## How was this patch tested?
Unit test
Compile with different commands:
```
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
```
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21838 from gengliangwang/from_and_to_avro.
## What changes were proposed in this pull request?
I propose to add new option for AVRO datasource which should control ignoring of files without `.avro` extension in read. The option has name `ignoreExtension` with default value `true`. If both options `ignoreExtension` and `avro.mapred.ignore.inputs.without.extension` are set, `ignoreExtension` overrides the former one. Here is an example of usage:
```
spark
.read
.option("ignoreExtension", false)
.avro("path to avro files")
```
## How was this patch tested?
I added a test which checks the option directly and a test for checking that new option overrides hadoop's config.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes#21798 from MaxGekk/avro-ignore-extension.
## What changes were proposed in this pull request?
Previously in the refactoring of Avro Serializer and Deserializer, a new class SerializableSchema is created for serializing the Avro schema:
https://github.com/apache/spark/pull/21762/files#diff-01fea32e6ec6bcf6f34d06282e08705aR37
On second thought, we can use `toString` method for serialization. After that, parse the JSON format schema on executor. This makes the code much simpler.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21829 from gengliangwang/removeSerializableSchema.
## What changes were proposed in this pull request?
Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value.
Add a new function to_avro for converting a column into binary of avro format with the specified schema.
This PR is in progress. Will add test cases.
## How was this patch tested?
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21774 from gengliangwang/from_and_to_avro.
## What changes were proposed in this pull request?
As stated in https://github.com/apache/spark/pull/21321, in the error messages we should use `catalogString`. This is not the case, as SPARK-22893 used `simpleString` in order to have the same representation everywhere and it missed some places.
The PR unifies the messages using alway the `catalogString` representation of the dataTypes in the messages.
## How was this patch tested?
existing/modified UTs
Author: Marco Gaido <marcogaido91@gmail.com>
Closes#21804 from mgaido91/SPARK-24268_catalog.
## What changes were proposed in this pull request?
In the PR, I propose to put all `Avro` options in new class `AvroOptions` in the same way as for other datasources `JSON` and `CSV`.
## How was this patch tested?
It was tested by `AvroSuite`
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes#21810 from MaxGekk/avro-options.
## What changes were proposed in this pull request?
This pr fixes lint-java and Scala 2.12 build.
lint-java:
```
[ERROR] src/test/resources/log4j.properties:[0] (misc) NewlineAtEndOfFile: File does not end with a newline.
```
Scala 2.12 build:
```
[error] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala:121: overloaded method value addTaskCompletionListener with alternatives:
[error] (f: org.apache.spark.TaskContext => Unit)org.apache.spark.TaskContext <and>
[error] (listener: org.apache.spark.util.TaskCompletionListener)org.apache.spark.TaskContext
[error] cannot be applied to (org.apache.spark.TaskContext => java.util.List[Runnable])
[error] context.addTaskCompletionListener { ctx =>
[error] ^
```
## How was this patch tested?
Manually executed lint-java and Scala 2.12 build in my local environment.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#21801 from ueshin/issues/SPARK-24386_24768/fix_build.
## What changes were proposed in this pull request?
In the PR, I propose to change default behaviour of AVRO datasource which currently ignores files without `.avro` extension in read by default. This PR sets the default value for `avro.mapred.ignore.inputs.without.extension` to `false` in the case if the parameter is not set by an user.
## How was this patch tested?
Added a test file without extension in AVRO format, and new test for reading the file with and wihout specified schema.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>
Closes#21769 from MaxGekk/avro-without-extension.
## What changes were proposed in this pull request?
In the PR, I propose to move `testFile()` to the common trait `SQLTestUtilsBase` and wrap test files in `AvroSuite` by the method `testFile()` which returns full paths to test files in the resource folder.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes#21773 from MaxGekk/test-file.
## What changes were proposed in this pull request?
Currently the Avro Deserializer converts input Avro format data to `Row`, and then convert the `Row` to `InternalRow`.
While the Avro Serializer converts `InternalRow` to `Row`, and then output Avro format data.
This PR allows direct conversion between `InternalRow` and Avro format data.
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21762 from gengliangwang/avro_io.
## What changes were proposed in this pull request?
Improve Avro unit test:
1. use QueryTest/SharedSQLContext/SQLTestUtils, instead of the duplicated test utils.
2. replace deprecated methods
This is a follow up PR for #21760, the PR passes pull request tests but failed in: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7842/
This PR is to fix it.
## How was this patch tested?
Unit test.
Compile with different commands:
```
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile
```
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21768 from gengliangwang/improve_avro_test.
## What changes were proposed in this pull request?
Improve Avro unit test:
1. use QueryTest/SharedSQLContext/SQLTestUtils, instead of the duplicated test utils.
2. replace deprecated methods
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21760 from gengliangwang/improve_avro_test.
We have hundreds of kafka topics need to be consumed in one application. The application master will throw OOM exception after hanging for nearly half of an hour.
OOM happens in the env with a lot of topics, and it's not convenient to set up such kind of env in the unit test. So I didn't change/add test case.
Author: Yuanbo Liu <yuanbo@Yuanbos-MacBook-Air.local>
Author: yuanbo <yuanbo@apache.org>
Closes#21690 from yuanboliu/master.
## What changes were proposed in this pull request?
Apache Avro (https://avro.apache.org) is a popular data serialization format. It is widely used in the Spark and Hadoop ecosystem, especially for Kafka-based data pipelines. Using the external package https://github.com/databricks/spark-avro, Spark SQL can read and write the avro data. Making spark-Avro built-in can provide a better experience for first-time users of Spark SQL and structured streaming. We expect the built-in Avro data source can further improve the adoption of structured streaming.
The proposal is to inline code from spark-avro package (https://github.com/databricks/spark-avro). The target release is Spark 2.4.
[Built-in AVRO Data Source In Spark 2.4.pdf](https://github.com/apache/spark/files/2181511/Built-in.AVRO.Data.Source.In.Spark.2.4.pdf)
## How was this patch tested?
Unit test
Author: Gengliang Wang <gengliang.wang@databricks.com>
Closes#21742 from gengliangwang/export_avro.
Setting the timestamp directly would cause exception on reading stream, it can be set directly only if the mode is not AT_TIMESTAMP
## What changes were proposed in this pull request?
The last patch in the kinesis streaming receiver sets the timestamp for the mode AT_TIMESTAMP, but this mode can only be set via the
`baseClientLibConfiguration.withTimestampAtInitialPositionInStream()
`
and can't be set directly using
`.withInitialPositionInStream()`
This patch fixes the issue.
## How was this patch tested?
Kinesis Receiver doesn't expose the internal state outside, so couldn't find the right way to test this change. Seeking for tips from other contributors here.
Author: Yash Sharma <ysharma@atlassian.com>
Closes#21541 from yashs360/ysharma/fix_kinesis_bug.
## What changes were proposed in this pull request?
SPARK-22893 tried to unify error messages about dataTypes. Unfortunately, still many places were missing the `simpleString` method in other to have the same representation everywhere.
The PR unified the messages using alway the simpleString representation of the dataTypes in the messages.
## How was this patch tested?
existing/modified UTs
Author: Marco Gaido <marcogaido91@gmail.com>
Closes#21321 from mgaido91/SPARK-24268.
This passes the unique task attempt id instead of attempt number to v2 data sources because attempt number is reused when stages are retried. When attempt numbers are reused, sources that track data by partition id and attempt number may incorrectly clean up data because the same attempt number can be both committed and aborted.
For v1 / Hadoop writes, generate a unique ID based on available attempt numbers to avoid a similar problem.
Closes#21558
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Ryan Blue <blue@apache.org>
Closes#21606 from vanzin/SPARK-24552.2.