Commit graph

524 commits

Author SHA1 Message Date
Gabor Somogyi d0443a74d1 [SPARK-26766][CORE] Remove the list of filesystems from HadoopDelegationTokenProvider.obtainDelegationTokens
## What changes were proposed in this pull request?

Delegation token providers interface now has a parameter `fileSystems` but this is needed only for `HadoopFSDelegationTokenProvider`.

In this PR I've addressed this issue in the following way:
* Removed `fileSystems` parameter from `HadoopDelegationTokenProvider`
* Moved `YarnSparkHadoopUtil.hadoopFSsToAccess` into `HadoopFSDelegationTokenProvider`
* Moved `spark.yarn.stagingDir` into core
* Moved `spark.yarn.access.namenodes` into core and renamed to `spark.kerberos.access.namenodes`
* Moved `spark.yarn.access.hadoopFileSystems` into core and renamed to `spark.kerberos.access.hadoopFileSystems`

## How was this patch tested?

Existing unit tests.

Closes #23698 from gaborgsomogyi/SPARK-26766.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-08 13:41:52 -08:00
Gengliang Wang 308996bc72 [SPARK-26716][SPARK-26765][FOLLOWUP][SQL] Clean up schema validation methods and override toString method in Avro
## What changes were proposed in this pull request?

In #23639, the API `supportDataType` is refactored. We should also remove the method `verifyWriteSchema` and `verifyReadSchema` in `DataSourceUtils`.

Since the error message use `FileFormat.toString` to specify the data source naming,  this PR also overriding the `toString` method in `AvroFileFormat`.

## How was this patch tested?

Unit test.

Closes #23699 from gengliangwang/SPARK-26716-followup.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 15:44:44 +08:00
ryne.yang fbc3c5e8a3
[SPARK-26718][SS] Fixed integer overflow in SS kafka rateLimit calculation
## What changes were proposed in this pull request?

Fix the integer overflow issue in rateLimit.

## How was this patch tested?

Pass the Jenkins with newly added UT for the possible case where integer could be overflowed.

Closes #23666 from linehrr/master.

Authored-by: ryne.yang <ryne.yang@acuityads.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-29 10:58:10 -08:00
Gengliang Wang 1beed0d7c2 [SPARK-26765][SQL] Avro: Validate input and output schema
## What changes were proposed in this pull request?

The API `supportDataType` in `FileFormat` helps to validate the output/input schema before exection starts. So that we can avoid some invalid data source IO, and users can see clean error messages.

This PR is to override the validation API in Avro data source.
Also, as per the spec of Avro(https://avro.apache.org/docs/1.8.2/spec.html), `NullType` is supported. This PR fixes the handling of `NullType`.

## How was this patch tested?

Unit test

Closes #23684 from gengliangwang/avroSupportDataType.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-30 00:17:33 +08:00
Wenchen Fan e97ab1d980 [SPARK-26695][SQL] data source v2 API refactor - continuous read
## What changes were proposed in this pull request?

Following https://github.com/apache/spark/pull/23430, this PR does the API refactor for continuous read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

The major changes:
1. rename `XXXContinuousReadSupport` to `XXXContinuousStream`
2. at the beginning of continuous streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.
3. remove all the hacks as we have finished all the read side API refactor

## How was this patch tested?

existing tests

Closes #23619 from cloud-fan/continuous.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-29 00:07:27 -08:00
Gabor Somogyi 773efede20 [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core.
## What changes were proposed in this pull request?

There are ugly provided dependencies inside core for the following:
* Hive
* Kafka

In this PR I've extracted them out. This PR contains the following:
* Token providers are now loaded with service loader
* Hive token provider moved to hive project
* Kafka token provider extracted into a new project

## How was this patch tested?

Existing + newly added unit tests.
Additionally tested on cluster.

Closes #23499 from gaborgsomogyi/SPARK-26254.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-25 10:36:00 -08:00
Wenchen Fan 098a2c41fc [SPARK-26520][SQL] data source v2 API refactor (micro-batch read)
## What changes were proposed in this pull request?

Following https://github.com/apache/spark/pull/23086, this PR does the API refactor for micro-batch read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

The major changes:
1. rename `XXXMicroBatchReadSupport` to `XXXMicroBatchReadStream`
2. implement `TableProvider`, `Table`, `ScanBuilder` and `Scan` for streaming sources
3. at the beginning of micro-batch streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.

followup:
support operator pushdown for stream sources

## How was this patch tested?

existing tests

Closes #23430 from cloud-fan/micro-batch.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-21 14:29:12 -08:00
Jungtaek Lim (HeartSaVioR) 2ebb79b2a6
[SPARK-26350][FOLLOWUP] Add actual verification on new UT introduced on SPARK-26350
## What changes were proposed in this pull request?

This patch adds the check to verify consumer group id is given correctly when custom group id is provided to Kafka parameter.

## How was this patch tested?

Modified UT.

Closes #23544 from HeartSaVioR/SPARK-26350-follow-up-actual-verification-on-UT.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-01-15 14:21:51 -08:00
Shixiong Zhu bafc7ac025
[SPARK-26350][SS] Allow to override group id of the Kafka consumer
## What changes were proposed in this pull request?

This PR allows the user to override `kafka.group.id` for better monitoring or security. The user needs to make sure there are not multiple queries or sources using the same group id.

It also fixes a bug that the `groupIdPrefix` option cannot be retrieved.

## How was this patch tested?

The new added unit tests.

Closes #23301 from zsxwing/SPARK-26350.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-01-14 13:37:24 -08:00
Oleksii Shkarupin 5b37092311
[SPARK-26538][SQL] Set default precision and scale for elements of postgres numeric array
## What changes were proposed in this pull request?

When determining CatalystType for postgres columns with type `numeric[]` set the type of array element to `DecimalType(38, 18)` instead of `DecimalType(0,0)`.

## How was this patch tested?

Tested with modified `org.apache.spark.sql.jdbc.JDBCSuite`.
Ran the `PostgresIntegrationSuite` manually.

Closes #23456 from a-shkarupin/postgres_numeric_array.

Lead-authored-by: Oleksii Shkarupin <a.shkarupin@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-12 11:06:39 -08:00
Dongjoon Hyun 5969b8a2ed
[SPARK-26541][BUILD] Add -Pdocker-integration-tests to dev/scalastyle
## What changes were proposed in this pull request?

This PR makes `scalastyle` to check `docker-integration-tests` module additionally and fixes one error.

## How was this patch tested?

Pass the Jenkins with the updated Scalastyle.
```
========================================================================
Running Scala style checks
========================================================================
Scalastyle checks passed.
```

Closes #23459 from dongjoon-hyun/SPARK-26541.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-05 00:55:17 -08:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
Dongjoon Hyun ceff0c8450
[SPARK-26428][SS][TEST] Minimize deprecated ProcessingTime usage
## What changes were proposed in this pull request?

Use of `ProcessingTime` class was deprecated in favor of `Trigger.ProcessingTime` in Spark 2.2. And, [SPARK-21464](https://issues.apache.org/jira/browse/SPARK-21464) minimized it at 2.2.1. Recently, it grows again in test suites. This PR aims to clean up newly introduced deprecation warnings for Spark 3.0.

## How was this patch tested?

Pass the Jenkins with existing tests and manually check the warnings.

Closes #23367 from dongjoon-hyun/SPARK-26428.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-22 00:43:59 -08:00
Shixiong Zhu 8e76d6621a
[SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka
## What changes were proposed in this pull request?

Due to [KAFKA-7703](https://issues.apache.org/jira/browse/KAFKA-7703), Kafka may return an earliest offset when we are request a latest offset. This will cause Spark to reprocess data.

As per suggestion in KAFKA-7703, we put a position call between poll and seekToEnd to block the fetch request triggered by `poll` before calling `seekToEnd`.

In addition, to avoid other unknown issues, we also use the previous known offsets to audit the latest offsets returned by Kafka. If we find some incorrect offsets (a latest offset is less than an offset in `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times.

## How was this patch tested?

Jenkins

Closes #23324 from zsxwing/SPARK-26267.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-12-21 10:41:25 -08:00
Vaclav Kosar 81d377d772 [SPARK-24933][SS] Report numOutputRows in SinkProgress
## What changes were proposed in this pull request?

SinkProgress should report similar properties like SourceProgress as long as they are available for given Sink. Count of written rows is metric availble for all Sinks. Since relevant progress information is with respect to commited rows, ideal object to carry this info is WriterCommitMessage. For brevity the implementation will focus only on Sinks with API V2 and on Micro Batch mode. Implemention for Continuous mode will be provided at later date.

### Before
```
{"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317"}
```

### After
```
{"description":"org.apache.spark.sql.kafka010.KafkaSourceProvider3c0bd317","numOutputRows":5000}
```

### This PR is related to:
- https://issues.apache.org/jira/browse/SPARK-24647
- https://issues.apache.org/jira/browse/SPARK-21313

## How was this patch tested?

Existing and new unit tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #21919 from vackosar/feature/SPARK-24933-numOutputRows.

Lead-authored-by: Vaclav Kosar <admin@vaclavkosar.com>
Co-authored-by: Kosar, Vaclav: Functions Transformation <Vaclav.Kosar@barclayscapital.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-12-17 11:50:24 -08:00
Gabor Somogyi 5a116e669c
[SPARK-26371][SS] Increase kafka ConfigUpdater test coverage.
## What changes were proposed in this pull request?

As Kafka delegation token added logic into ConfigUpdater it would be good to test it.
This PR contains the following changes:
* ConfigUpdater extracted to a separate file and renamed to KafkaConfigUpdater
* mockito-core dependency added to kafka-0-10-sql
* Unit tests added

## How was this patch tested?

Existing + new unit tests + on cluster.

Closes #23321 from gaborgsomogyi/SPARK-26371.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-17 10:07:35 -08:00
jasonwayne 9c481c7a6b [SPARK-26360] remove redundant validateQuery call
## What changes were proposed in this pull request?
remove a redundant `KafkaWriter.validateQuery` call in `KafkaSourceProvider `

## How was this patch tested?
Just removing duplicate codes, so I just build and run unit tests.

Closes #23309 from JasonWayne/SPARK-26360.

Authored-by: jasonwayne <wuwenjie0102@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-14 10:47:58 +08:00
Gabor Somogyi 6daa783094 [SPARK-26322][SS] Add spark.kafka.sasl.token.mechanism to ease delegation token configuration.
## What changes were proposed in this pull request?

When Kafka delegation token obtained, SCRAM `sasl.mechanism` has to be configured for authentication. This can be configured on the related source/sink which is inconvenient from user perspective. Such granularity is not required and this configuration can be implemented with one central parameter.

In this PR `spark.kafka.sasl.token.mechanism` added to configure this centrally (default: `SCRAM-SHA-512`).

## How was this patch tested?

Existing unit tests + on cluster.

Closes #23274 from gaborgsomogyi/SPARK-26322.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-12 16:45:50 -08:00
Gabor Somogyi 9b1f6c8bab [SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter
## What changes were proposed in this pull request?

spark.kafka.sasl.kerberos.service.name is an optional parameter but most of the time value `kafka` has to be set. As I've written in the jira the following reasoning is behind:
* Kafka's configuration guide suggest the same value: https://kafka.apache.org/documentation/#security_sasl_kerberos_brokerconfig
* It would be easier for spark users by providing less configuration
* Other streaming engines are doing the same

In this PR I've changed the parameter from optional to `WithDefault` and set `kafka` as default value.

## How was this patch tested?

Available unit tests + on cluster.

Closes #23254 from gaborgsomogyi/SPARK-26304.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-07 13:58:02 -08:00
Wenchen Fan 2b2c94a3ee [SPARK-25528][SQL] data source v2 API refactor (batch read)
## What changes were proposed in this pull request?

This is the first step of the data source v2 API refactor [proposal](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

It adds the new API for batch read, without removing the old APIs, as they are still needed for streaming sources.

More concretely, it adds
1. `TableProvider`, works like an anonymous catalog
2. `Table`, represents a structured data set.
3. `ScanBuilder` and `Scan`, a logical represents of data source scan
4. `Batch`, a physical representation of data source batch scan.

## How was this patch tested?

existing tests

Closes #23086 from cloud-fan/refactor-batch.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-30 00:02:43 -08:00
Gabor Somogyi 0166c7373e [SPARK-25501][SS] Add kafka delegation token support.
## What changes were proposed in this pull request?

It adds kafka delegation token support for structured streaming. Please see the relevant [SPIP](https://docs.google.com/document/d/1ouRayzaJf_N5VQtGhVq9FURXVmRpXzEEWYHob0ne3NY/edit?usp=sharing)

What this PR contains:
* Configuration parameters for the feature
* Delegation token fetching from broker
* Usage of token through dynamic JAAS configuration
* Minor refactoring in the existing code

What this PR doesn't contain:
* Documentation changes because design can change

## How was this patch tested?

Existing tests + added small amount of additional unit tests.

Because it's an external service integration mainly tested on cluster.
* 4 node cluster
* Kafka broker version 1.1.0
* Topic with 4 partitions
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-256

An example of obtaining a token:
```
18/10/01 01:07:49 INFO kafka010.TokenUtil: TOKENID         HMAC                           OWNER           RENEWERS                  ISSUEDATE       EXPIRYDATE      MAXDATE
18/10/01 01:07:49 INFO kafka010.TokenUtil: D1-v__Q5T_uHx55rW16Jwg [hidden] User:user    []                        2018-10-01T01:07 2018-10-02T01:07 2018-10-08T01:07
18/10/01 01:07:49 INFO security.KafkaDelegationTokenProvider: Get token from Kafka: Kind: KAFKA_DELEGATION_TOKEN, Service: kafka.server.delegation.token, Ident: 44 31 2d 76 5f 5f 51 35 54 5f 75 48 78 35 35 72 57 31 36 4a 77 67
```

An example token usage:
```
18/10/01 01:08:07 INFO kafka010.KafkaSecurityHelper: Scram JAAS params: org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="D1-v__Q5T_uHx55rW16Jwg" password="[hidden]";
18/10/01 01:08:07 INFO kafka010.KafkaSourceProvider: Delegation token detected, using it for login.
```

Closes #22598 from gaborgsomogyi/SPARK-25501.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 18:00:47 -08:00
Wenchen Fan fa0d4bf699 [SPARK-25829][SQL] remove duplicated map keys with last wins policy
## What changes were proposed in this pull request?

Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.

This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.

updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.

For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.

## How was this patch tested?

updated tests and new tests

Closes #23124 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 23:42:13 +08:00
Anastasios Zouzias 2512a1d429 [SPARK-26121][STRUCTURED STREAMING] Allow users to define prefix of Kafka's consumer group (group.id)
## What changes were proposed in this pull request?

Allow the Spark Structured Streaming user to specify the prefix of the consumer group (group.id), compared to force consumer group ids of the form `spark-kafka-source-*`

## How was this patch tested?

Unit tests provided by Spark (backwards compatible change, i.e., user can optionally use the functionality)

`mvn test -pl external/kafka-0-10`

Closes #23103 from zouzias/SPARK-26121.

Authored-by: Anastasios Zouzias <anastasios@sqooba.io>
Signed-off-by: cody koeninger <cody@koeninger.org>
2018-11-26 11:10:38 -06:00
Dongjoon Hyun 1d3dd58d21
[SPARK-25954][SS][FOLLOWUP][TEST-MAVEN] Add Zookeeper 3.4.7 test dependency to Kafka modules
## What changes were proposed in this pull request?

This is a followup of #23099 . After upgrading to Kafka 2.1.0, maven test fails due to Zookeeper test dependency while sbt test succeeds.

- [sbt test on master branch](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/5203/)
- [maven test on master branch](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5653/)

The root cause is that the embedded Kafka server is using [Zookeepr 3.4.7 API](https://zookeeper.apache.org/doc/r3.4.7/api/org/apache/zookeeper/AsyncCallback.MultiCallback.html
) while Apache Spark provides Zookeeper 3.4.6. This PR adds a test dependency.

```
KafkaMicroBatchV2SourceSuite:
*** RUN ABORTED ***
...
org.apache.spark.sql.kafka010.KafkaTestUtils.setupEmbeddedKafkaServer(KafkaTestUtils.scala:123)
  ...
  Cause: java.lang.ClassNotFoundException: org.apache.zookeeper.AsyncCallback$MultiCallback
  at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
  at kafka.zk.KafkaZkClient$.apply(KafkaZkClient.scala:1693)
  at kafka.server.KafkaServer.createZkClient$1(KafkaServer.scala:348)
  at kafka.server.KafkaServer.initZkClient(KafkaServer.scala:372)
  at kafka.server.KafkaServer.startup(KafkaServer.scala:202)
  at org.apache.spark.sql.kafka010.KafkaTestUtils.$anonfun$setupEmbeddedKafkaServer$2(KafkaTestUtils.scala:120)
  at org.apache.spark.sql.kafka010.KafkaTestUtils.$anonfun$setupEmbeddedKafkaServer$2$adapted(KafkaTestUtils.scala:116)
  ...
```

## How was this patch tested?

Pass the maven Jenkins test.

Closes #23119 from dongjoon-hyun/SPARK-25954-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 22:45:08 -08:00
Dongjoon Hyun a480a62563
[SPARK-25954][SS] Upgrade to Kafka 2.1.0
## What changes were proposed in this pull request?

[Kafka 2.1.0 vote](https://lists.apache.org/thread.html/9f487094491e512b556a1c9c3c6034ac642b088e3f797e3d192ebc9d%3Cdev.kafka.apache.org%3E) passed. Since Kafka 2.1.0 includes official JDK 11 support [KAFKA-7264](https://issues.apache.org/jira/browse/KAFKA-7264), we had better use that.

## How was this patch tested?

Pass the Jenkins.

Closes #23099 from dongjoon-hyun/SPARK-25954.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-21 00:24:34 -08:00
Sean Owen 32365f8177 [SPARK-26090][CORE][SQL][ML] Resolve most miscellaneous deprecation and build warnings for Spark 3
## What changes were proposed in this pull request?

The build has a lot of deprecation warnings. Some are new in Scala 2.12 and Java 11. We've fixed some, but I wanted to take a pass at fixing lots of easy miscellaneous ones here.

They're too numerous and small to list here; see the pull request. Some highlights:

- `BeanInfo` is deprecated in 2.12, and BeanInfo classes are pretty ancient in Java. Instead, case classes can explicitly declare getters
- Eta expansion of zero-arg methods; foo() becomes () => foo() in many cases
- Floating-point Range is inexact and deprecated, like 0.0 to 100.0 by 1.0
- finalize() is finally deprecated (just needs to be suppressed)
- StageInfo.attempId was deprecated and easiest to remove here

I'm not now going to touch some chunks of deprecation warnings:

- Parquet deprecations
- Hive deprecations (particularly serde2 classes)
- Deprecations in generated code (mostly Thriftserver CLI)
- ProcessingTime deprecations (we may need to revive this class as internal)
- many MLlib deprecations because they concern methods that may be removed anyway
- a few Kinesis deprecations I couldn't figure out
- Mesos get/setRole, which I don't know well
- Kafka/ZK deprecations (e.g. poll())
- Kinesis
- a few other ones that will probably resolve by deleting a deprecated method

## How was this patch tested?

Existing tests, including manual testing with the 2.11 build and Java 11.

Closes #23065 from srowen/SPARK-26090.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 09:16:42 -06:00
Sean Owen 630e25e355 [SPARK-26026][BUILD] Published Scaladoc jars missing from Maven Central
## What changes were proposed in this pull request?

This restores scaladoc artifact generation, which got dropped with the Scala 2.12 update. The change looks large, but is almost all due to needing to make the InterfaceStability annotations top-level classes (i.e. `InterfaceStability.Stable` -> `Stable`), unfortunately. A few inner class references had to be qualified too.

Lots of scaladoc warnings now reappear. We can choose to disable generation by default and enable for releases, later.

## How was this patch tested?

N/A; build runs scaladoc now.

Closes #23069 from srowen/SPARK-26026.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 08:06:33 -06:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Yuanjian Li 2977e2312d [SPARK-25986][BUILD] Add rules to ban throw Errors in application code
## What changes were proposed in this pull request?

Add scala and java lint check rules to ban the usage of `throw new xxxErrors` and fix up all exists instance followed by https://github.com/apache/spark/pull/22989#issuecomment-437939830. See more details in https://github.com/apache/spark/pull/22969.

## How was this patch tested?

Local test with lint-scala and lint-java.

Closes #22989 from xuanyuanking/SPARK-25986.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 13:05:18 -08:00
Gengliang Wang 922dfe4865
[SPARK-25965][SQL][TEST] Add avro read benchmark
Add read benchmark for Avro, which is missing for a period.
The benchmark is similar to `DataSourceReadBenchmark` and `OrcReadBenchmark`

Manually run benchmark

Closes #22966 from gengliangwang/avroReadBenchmark.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 11:26:26 -08:00
Sean Owen 2d085c13b7 [SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request?

Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent

## How was this patch tested?

Existing tests.

Closes #22988 from srowen/SPARK-25984.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 09:52:14 -06:00
Gengliang Wang 57eddc7182 [SPARK-25886][SQL][MINOR] Improve error message of FailureSafeParser and from_avro in FAILFAST mode
## What changes were proposed in this pull request?

Currently in `FailureSafeParser` and `from_avro`, the exception is created with such code
```
throw new SparkException("Malformed records are detected in record parsing. " +
s"Parse Mode: ${FailFastMode.name}.", e.cause)
```

1. The cause part should be `e` instead of `e.cause`
2. If `e` contains non-null message, it should be shown in `from_json`/`from_csv`/`from_avro`, e.g.
```
com.fasterxml.jackson.core.JsonParseException: Unexpected character ('1' (code 49)): was expecting a colon to separate field name and value
at [Source: (InputStreamReader); line: 1, column: 7]
```
3.Kindly show hint for trying PERMISSIVE in error message.

## How was this patch tested?
Unit test.

Closes #22895 from gengliangwang/improve_error_msg.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-31 20:22:57 +08:00
yucai f8484e49ef
[SPARK-25663][SPARK-25661][SQL][TEST] Refactor BuiltInDataSourceWriteBenchmark, DataSourceWriteBenchmark and AvroWriteBenchmark to use main method
## What changes were proposed in this pull request?

Refactor BuiltInDataSourceWriteBenchmark, DataSourceWriteBenchmark and AvroWriteBenchmark to use main method.

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.BuiltInDataSourceWriteBenchmark"

SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "avro/test:runMain org.apache.spark.sql.execution.benchmark.AvroWriteBenchmark"
```
## How was this patch tested?

manual tests

Closes #22861 from yucai/BuiltInDataSourceWriteBenchmark.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 03:03:42 -07:00
Dilip Biswal 34c3bc9f1e
[SPARK-25618][SQL][TEST] Reduce time taken to execute KafkaContinuousSourceStressForDontFailOnDataLossSuite
## What changes were proposed in this pull request?
In this test, i have reduced the test time to 20 secs from 1 minute while reducing the sleep time from 1 sec to 100 milliseconds.

With this change, i was able to run the test in 20+ seconds consistently on my laptop. I would like see if it passes in jenkins consistently.

## How was this patch tested?
Its a test fix.

Closes #22900 from dilipbiswal/SPARK-25618.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 02:57:39 -07:00
Gengliang Wang 0ad93b0931 [SPARK-25883][SQL][MINOR] Override method prettyName in from_avro/to_avro
## What changes were proposed in this pull request?

Previously in from_avro/to_avro, we override the method `simpleString` and `sql` for the string output. However, the override only affects the alias naming:
```
Project [from_avro('col,
...
, (mode,PERMISSIVE)) AS from_avro(col, struct<col1:bigint,col2:double>, Map(mode -> PERMISSIVE))#11]
```
It only makes the alias name quite long: `from_avro(col, struct<col1:bigint,col2:double>, Map(mode -> PERMISSIVE))`).

We should follow `from_csv`/`from_json` here, to override the method prettyName only, and we will get a clean alias name

```
... AS from_avro(col)#11
```

## How was this patch tested?

Manual check

Closes #22890 from gengliangwang/revise_from_to_avro.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-30 23:59:37 -07:00
Gengliang Wang 24e8c27dfe [SPARK-25819][SQL] Support parse mode option for the function from_avro
## What changes were proposed in this pull request?

Current the function `from_avro` throws exception on reading corrupt records.
In practice, there could be various reasons of data corruption. It would be good to support `PERMISSIVE` mode and allow the function from_avro to process all the input file/streaming, which is consistent with from_json and from_csv. There is no obvious down side for supporting `PERMISSIVE` mode.

Different from `from_csv` and `from_json`, the default parse mode is `FAILFAST` for the following reasons:
1. Since Avro is structured data format, input data is usually able to be parsed by certain schema.  In such case, exposing the problems of input data to users is better than hiding it.
2. For `PERMISSIVE` mode, we have to force the data schema as fully nullable. This seems quite unnecessary for Avro. Reversing non-null schema might archive more perf optimizations in Spark.
3. To be consistent with the behavior in Spark 2.4 .

## How was this patch tested?

Unit test

Manual previewing generated html for the Avro data source doc:

![image](https://user-images.githubusercontent.com/1097932/47510100-02558880-d8aa-11e8-9d57-a43daee4c6b9.png)

Closes #22814 from gengliangwang/improve_from_avro.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-26 11:39:38 +08:00
Sean Owen f83fedc9f2 [SPARK-25737][CORE] Remove JavaSparkContextVarargsWorkaround
## What changes were proposed in this pull request?

Remove JavaSparkContextVarargsWorkaround

## How was this patch tested?

Existing tests.

Closes #22729 from srowen/SPARK-25737.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-24 14:43:51 -05:00
Sean Owen 734c6af0dd [SPARK-24601][FOLLOWUP] Update Jackson to 2.9.6 in Kinesis
## What changes were proposed in this pull request?

Also update Kinesis SDK's Jackson to match Spark's

## How was this patch tested?

Existing tests, including Kinesis ones, which ought to be hereby triggered.
This was uncovered, I believe, in https://github.com/apache/spark/pull/22729#issuecomment-430666080

Closes #22757 from srowen/SPARK-24601.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-18 07:00:00 -05:00
Takeshi Yamamuro a9f685bb70 [SPARK-25734][SQL] Literal should have a value corresponding to dataType
## What changes were proposed in this pull request?
`Literal.value` should have a value a value corresponding to `dataType`. This pr added code to verify it and fixed the existing tests to do so.

## How was this patch tested?
Modified the existing tests.

Closes #22724 from maropu/SPARK-25734.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 11:02:39 +08:00
Dilip Biswal 9d4dd7992b [SPARK-25631][SPARK-25632][SQL][TEST] Improve the test runtime of KafkaRDDSuite
## What changes were proposed in this pull request?
Set a reasonable poll timeout thats used while consuming topics/partitions from kafka. In the
absence of it, a default of 2 minute is used as the timeout values. And all the negative tests take a minimum of 2 minute to execute.

After this change, we save about 4 minutes in this suite.

## How was this patch tested?
Test fix.

Closes #22670 from dilipbiswal/SPARK-25631.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-16 17:49:40 -05:00
Sean Owen 703e6da1ec [SPARK-25705][BUILD][STREAMING][TEST-MAVEN] Remove Kafka 0.8 integration
## What changes were proposed in this pull request?

Remove Kafka 0.8 integration

## How was this patch tested?

Existing tests, build scripts

Closes #22703 from srowen/SPARK-25705.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-16 09:10:24 -05:00
Gengliang Wang 2eaf058788 [SPARK-25718][SQL] Detect recursive reference in Avro schema and throw exception
## What changes were proposed in this pull request?

Avro schema allows recursive reference, e.g. the schema for linked-list in https://avro.apache.org/docs/1.8.2/spec.html#schema_record
```
{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],                      // old name for this
  "fields" : [
    {"name": "value", "type": "long"},             // each element has a long
    {"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}
```

In current Spark SQL, it is impossible to convert the schema as `StructType` . Run `SchemaConverters.toSqlType(avroSchema)` and we will get stack overflow exception.

We should detect the recursive reference and throw exception for it.
## How was this patch tested?

New unit test case.

Closes #22709 from gengliangwang/avroRecursiveRef.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-13 14:49:38 +08:00
Sean Owen a001814189 [SPARK-25598][STREAMING][BUILD][TEST-MAVEN] Remove flume connector in Spark 3
## What changes were proposed in this pull request?

Removes all vestiges of Flume in the build, for Spark 3.
I don't think this needs Jenkins config changes.

## How was this patch tested?

Existing tests.

Closes #22692 from srowen/SPARK-25598.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:28:06 -07:00
Dilip Biswal adf648b5be [SPARK-25615][SQL][TEST] Improve the test runtime of KafkaSinkSuite: streaming write to non-existing topic
## What changes were proposed in this pull request?
Specify `kafka.max.block.ms` to 10 seconds while creating the kafka writer. In the absence of this overridden config, by default it uses a default time out of 60 seconds.

With this change the test completes in close to 10 seconds as opposed to 1 minute.

## How was this patch tested?
This is a test fix.

Closes #22671 from dilipbiswal/SPARK-25615.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:10:07 -07:00
Dongjoon Hyun 9cbf105ab1
[SPARK-25644][SS][FOLLOWUP][BUILD] Fix Scala 2.12 build error due to foreachBatch
## What changes were proposed in this pull request?

This PR fixes the Scala-2.12 build error due to ambiguity in `foreachBatch` test cases.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/428/console
```scala
[error] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:102: ambiguous reference to overloaded definition,
[error] both method foreachBatch in class DataStreamWriter of type (function: org.apache.spark.api.java.function.VoidFunction2[org.apache.spark.sql.Dataset[Int],Long])org.apache.spark.sql.streaming.DataStreamWriter[Int]
[error] and  method foreachBatch in class DataStreamWriter of type (function: (org.apache.spark.sql.Dataset[Int], Long) => Unit)org.apache.spark.sql.streaming.DataStreamWriter[Int]
[error] match argument types ((org.apache.spark.sql.Dataset[Int], Any) => Unit)
[error]       ds.writeStream.foreachBatch((_, _) => {}).trigger(Trigger.Continuous("1 second")).start()
[error]                      ^
[error] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:106: ambiguous reference to overloaded definition,
[error] both method foreachBatch in class DataStreamWriter of type (function: org.apache.spark.api.java.function.VoidFunction2[org.apache.spark.sql.Dataset[Int],Long])org.apache.spark.sql.streaming.DataStreamWriter[Int]
[error] and  method foreachBatch in class DataStreamWriter of type (function: (org.apache.spark.sql.Dataset[Int], Long) => Unit)org.apache.spark.sql.streaming.DataStreamWriter[Int]
[error] match argument types ((org.apache.spark.sql.Dataset[Int], Any) => Unit)
[error]       ds.writeStream.foreachBatch((_, _) => {}).partitionBy("value").start()
[error]                      ^
```

## How was this patch tested?

Manual.

Since this failure occurs in Scala-2.12 profile and test cases, Jenkins will not test this. We need to build with Scala-2.12 and run the tests.

Closes #22649 from dongjoon-hyun/SPARK-SCALA212.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-06 09:40:42 -07:00
gatorsmile 7ef65c0537 [HOT-FIX] Fix compilation errors. 2018-10-06 08:50:50 -07:00
gatorsmile 44cf800c83 [SPARK-25655][BUILD] Add -Pspark-ganglia-lgpl to the scala style check.
## What changes were proposed in this pull request?
Our lint failed due to the following errors:
```
[INFO] --- scalastyle-maven-plugin:1.0.0:check (default)  spark-ganglia-lgpl_2.11 ---
error file=/home/jenkins/workspace/spark-master-maven-snapshots/spark/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala message=
      Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you
      should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead.
      If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with
      // scalastyle:off caselocale
      .toUpperCase
      .toLowerCase
      // scalastyle:on caselocale
     line=67 column=49
error file=/home/jenkins/workspace/spark-master-maven-snapshots/spark/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala message=
      Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you
      should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead.
      If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with
      // scalastyle:off caselocale
      .toUpperCase
      .toLowerCase
      // scalastyle:on caselocale
     line=71 column=32
Saving to outputFile=/home/jenkins/workspace/spark-master-maven-snapshots/spark/external/spark-ganglia-lgpl/target/scalastyle-output.xml
```

See https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-lint/8890/

## How was this patch tested?
N/A

Closes #22647 from gatorsmile/fixLint.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-06 14:25:48 +08:00
Gengliang Wang 928d0739c4 [SPARK-25595] Ignore corrupt Avro files if flag IGNORE_CORRUPT_FILES enabled
## What changes were proposed in this pull request?

With flag `IGNORE_CORRUPT_FILES` enabled, schema inference should ignore corrupt Avro files, which is consistent with Parquet and Orc data source.

## How was this patch tested?

Unit test

Closes #22611 from gengliangwang/ignoreCorruptAvro.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-03 17:08:55 +08:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
seancxmao 21f0b73dbc [SPARK-25453][SQL][TEST][.FFFFFFFFF] OracleIntegrationSuite IllegalArgumentException: Timestamp format must be yyyy-mm-dd hh:mm:ss
## What changes were proposed in this pull request?
This PR aims to fix the failed test of `OracleIntegrationSuite`.

## How was this patch tested?
Existing integration tests.

Closes #22461 from seancxmao/SPARK-25453.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-30 22:49:14 -07:00