Commit graph

670 commits

Author SHA1 Message Date
Burak Yavuz 1cd19ad92d [SPARK-30669][SS] Introduce AdmissionControl APIs for StructuredStreaming
### What changes were proposed in this pull request?

We propose to add a new interface `SupportsAdmissionControl` and `ReadLimit`. A ReadLimit defines how much data should be read in the next micro-batch. `SupportsAdmissionControl` specifies that a source can rate limit its ingest into the system. The source can tell the system what the user specified as a read limit, and the system can enforce this limit within each micro-batch or impose its own limit if the Trigger is Trigger.Once() for example.

We then use this interface in FileStreamSource, KafkaSource, and KafkaMicroBatchStream.

### Why are the changes needed?

Sources currently have no information around execution semantics such as whether the stream is being executed in Trigger.Once() mode. This interface will pass this information into the sources as part of planning. With a trigger like Trigger.Once(), the semantics are to process all the data available to the datasource in a single micro-batch. However, this semantic can be broken when data source options such as `maxOffsetsPerTrigger` (in the Kafka source) rate limit the amount of data read for that micro-batch without this interface.

### Does this PR introduce any user-facing change?

DataSource developers can extend this interface for their streaming sources to add admission control into their system and correctly support Trigger.Once().

### How was this patch tested?

Existing tests, as this API is mostly internal

Closes #27380 from brkyvz/rateLimit.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-30 22:02:48 -08:00
Wenchen Fan 9f42be25eb [SPARK-29665][SQL] refine the TableProvider interface
### What changes were proposed in this pull request?

Instead of having several overloads of `getTable` method in `TableProvider`, it's better to have 2 methods explicitly: `inferSchema` and `inferPartitioning`. With a single `getTable` method that takes everything: schema, partitioning and properties.

This PR also adds a `supportsExternalMetadata` method in `TableProvider`, to indicate if the source support external table metadata. If this flag is false:
1. spark.read.schema... is disallowed and fails
2. when we support creating v2 tables in session catalog,  spark only keeps table properties in the catalog.

### Why are the changes needed?

API improvement.

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

existing tests

Closes #26868 from cloud-fan/provider2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 13:37:43 +08:00
Shixiong Zhu f56ba37d8b
[SPARK-30656][SS] Support the "minPartitions" option in Kafka batch source and streaming source v1
### What changes were proposed in this pull request?

- Add `minPartitions` support for Kafka Streaming V1 source.
- Add `minPartitions` support for Kafka batch V1  and V2 source.
- There is lots of refactoring (moving codes to KafkaOffsetReader) to reuse codes.

### Why are the changes needed?

Right now, the "minPartitions" option only works in Kafka streaming source v2. It would be great that we can support it in batch and streaming source v1 (v1 is the fallback mode when a user hits a regression in v2) as well.

### Does this PR introduce any user-facing change?

Yep. The `minPartitions` options is supported in Kafka batch and streaming queries for both data source V1 and V2.

### How was this patch tested?

New unit tests are added to test "minPartitions".

Closes #27388 from zsxwing/kafka-min-partitions.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-30 18:14:50 -08:00
Yuchen Huo d0800fc8e2 [SPARK-30314] Add identifier and catalog information to DataSourceV2Relation
### What changes were proposed in this pull request?

Add identifier and catalog information in DataSourceV2Relation so it would be possible to do richer checks in checkAnalysis step.

### Why are the changes needed?

In data source v2, table implementations are all customized so we may not be able to get the resolved identifier from tables them selves. Therefore we encode the table and catalog information in DSV2Relation so no external changes are needed to make sure this information is available.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Unit tests in the following suites:
CatalogManagerSuite.scala
CatalogV2UtilSuite.scala
SupportsCatalogOptionsSuite.scala
PlanResolutionSuite.scala

Closes #26957 from yuchenhuo/SPARK-30314.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-26 12:59:24 -08:00
Guy Khazma 2d59ca464e [SPARK-30475][SQL] File source V2: Push data filters for file listing
### What changes were proposed in this pull request?
Follow up on [SPARK-30428](https://github.com/apache/spark/pull/27112) which added support for partition pruning in File source V2.
This PR implements the necessary changes in order to pass the `dataFilters` to the `listFiles`. This enables having `FileIndex` implementations which use the `dataFilters` for further pruning the file listing (see the discussion [here](https://github.com/apache/spark/pull/27112#discussion_r364757217)).

### Why are the changes needed?
Datasources such as `csv` and `json` do not implement the `SupportsPushDownFilters` trait. In order to support data skipping uniformly for all file based data sources, one can override the `listFiles` method in a `FileIndex` implementation, which consults external metadata and prunes the list of files.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Modifying the unit tests for v2 file sources to verify the `dataFilters` are passed

Closes #27157 from guykhazma/PushdataFiltersInFileListing.

Authored-by: Guy Khazma <guykhag@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-20 20:20:37 -08:00
Maxim Gekk fd69533593 [SPARK-30482][CORE][SQL][TESTS][FOLLOW-UP] Output caller info in log appenders while reaching the limit
### What changes were proposed in this pull request?
In the PR, I propose to output additional msg from the tests where a log appender is added. The message is printed as a part of `IllegalStateException` in the case of reaching the limit of maximum number of logged events.

### Why are the changes needed?
If a log appender is not removed from the log4j appenders list. the caller message could help to investigate the problem and find the test which doesn't remove the log appender.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By running the modified test suites `AvroSuite`, `CSVSuite`, `ResolveHintsSuite` and etc.

Closes #27296 from MaxGekk/assign-name-to-log-appender.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-21 10:19:07 +09:00
Maxim Gekk ab048990e0 [SPARK-30558][SQL] Avoid rebuilding AvroOptions per each partition
### What changes were proposed in this pull request?
In the PR, I propose move out creation of `AvroOption` from `AvroPartitionReaderFactory.buildReader`, and create it earlier in `AvroScan.createReaderFactory`.

### Why are the changes needed?
- To avoid building `AvroOptions` from a map of Avro options and Hadoop conf per each partition.
- If an instance of `AvroOptions` is built only once at the driver side, we could output warnings while parsing Avro options and don't worry about noisiness of the warnings.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By `AvroSuite`

Closes #27272 from MaxGekk/avro-options-once-for-read.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-20 15:22:23 +09:00
Gabor Somogyi 6c178a5d16 [SPARK-30495][SS] Consider spark.security.credentials.kafka.enabled and cluster configuration when checking latest delegation token
### What changes were proposed in this pull request?
Spark SQL Kafka consumer connector considers delegation token usage even if the user configures `sasl.jaas.config` manually.

In this PR I've added `spark.security.credentials.kafka.enabled` and cluster configuration check to the condition.

### Why are the changes needed?
Now it's not possible to configure `sasl.jaas.config` manually.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existing + additional unit tests.

Closes #27191 from gaborgsomogyi/SPARK-30495.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 11:46:34 -08:00
Maxim Gekk 0f3d744c3f [MINOR][TESTS] Remove unsupported header option in AvroSuite
### What changes were proposed in this pull request?
In the PR, I propose to remove the `header` option in the `Avro source v2: support partition pruning` test.

### Why are the changes needed?
The option is not supported by Avro, and may misleading readers.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By `AvroSuite`.

Closes #27203 from MaxGekk/avro-suite-remove-header-option.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-14 11:49:50 -08:00
Maxim Gekk db7262a00f [SPARK-30509][SQL] Fix deprecation log warning in Avro schema inferring
### What changes were proposed in this pull request?
In the PR, I propose to check the `ignoreExtensionKey` option in the case insensitive map of `AvroOption`.

### Why are the changes needed?
The map `options` passed to `AvroUtils.inferSchema` contains all keys in the lower cases in fact. Actually, the map is converted from a `CaseInsensitiveStringMap`. Consequently, the check 3663dbe541/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala (L45) always return `false`, and the deprecation log warning is never printed.

### Does this PR introduce any user-facing change?
Yes, after the changes the log warning is printed once.

### How was this patch tested?
Added new test to `AvroSuite` which checks existence of log warning.

Closes #27200 from MaxGekk/avro-fix-ignoreExtension-contains.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-14 11:48:07 -08:00
Dongjoon Hyun 28fc0437ce [SPARK-28152][SQL][FOLLOWUP] Add a legacy conf for old MsSqlServerDialect numeric mapping
### What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/25248 .

### Why are the changes needed?

The new behavior cannot access the existing table which is created by old behavior.
This PR provides a way to avoid new behavior for the existing users.

### Does this PR introduce any user-facing change?

Yes. This will fix the broken behavior on the existing tables.

### How was this patch tested?

Pass the Jenkins and manually run JDBC integration test.
```
build/mvn install -DskipTests
build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test
```

Closes #27184 from dongjoon-hyun/SPARK-28152-CONF.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 23:03:34 -08:00
Maxim Gekk f5118f81e3 [SPARK-30409][SPARK-29173][SQL][TESTS] Use NoOp datasource in SQL benchmarks
### What changes were proposed in this pull request?
In the PR, I propose to replace `.collect()`, `.count()` and `.foreach(_ => ())` in SQL benchmarks and use the `NoOp` datasource. I added an implicit class to `SqlBasedBenchmark` with the `.noop()` method. It can be used in benchmark like: `ds.noop()`. The last one is unfolded to `ds.write.format("noop").mode(Overwrite).save()`.

### Why are the changes needed?
To avoid additional overhead that `collect()` (and other actions) has. For example, `.collect()` has to convert values according to external types and pull data to the driver. This can hide actual performance regressions or improvements of benchmarked operations.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Re-run all modified benchmarks using Amazon EC2.

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/10 |

- Run `TPCDSQueryBenchmark` using instructions from the PR #26049
```
# `spark-tpcds-datagen` needs this. (JDK8)
$ git clone https://github.com/apache/spark.git -b branch-2.4 --depth 1 spark-2.4
$ export SPARK_HOME=$PWD
$ ./build/mvn clean package -DskipTests

# Generate data. (JDK8)
$ git clone gitgithub.com:maropu/spark-tpcds-datagen.git
$ cd spark-tpcds-datagen/
$ build/mvn clean package
$ mkdir -p /data/tpcds
$ ./bin/dsdgen --output-location /data/tpcds/s1  // This need `Spark 2.4`
```
- Other benchmarks ran by the script:
```
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.AggregateBenchmark'],
    ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.BloomFilterBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.DataSourceReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.DateTimeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.ExtractBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.InExpressionBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.IntervalBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.JoinBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.MiscBenchmark'],
    ['hive/test', 'org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcNestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcV2NestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.ParquetNestedSchemaPruningBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.RangeBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.UDFBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideTableBenchmark'],
    ['hive/test', 'org.apache.spark.sql.hive.orc.OrcReadBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'],
    ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #27078 from MaxGekk/noop-in-benchmarks.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 13:18:19 -08:00
Burak Yavuz f8d59572b0 [SPARK-29219][SQL] Introduce SupportsCatalogOptions for TableProvider
### What changes were proposed in this pull request?

This PR introduces `SupportsCatalogOptions` as an interface for `TableProvider`. Through `SupportsCatalogOptions`, V2 DataSources can implement the two methods `extractIdentifier` and `extractCatalog` to support the creation, and existence check of tables without requiring a formal TableCatalog implementation.

We currently don't support all SaveModes for DataSourceV2 in DataFrameWriter.save. The idea here is that eventually File based tables can be written with `DataFrameWriter.save(path)` will create a PathIdentifier where the name is `path`, and the V2SessionCatalog will be able to perform FileSystem checks at `path` to support ErrorIfExists and Ignore SaveModes.

### Why are the changes needed?

To support all Save modes for V2 data sources with DataFrameWriter. Since we can now support table creation, we will be able to provide partitioning information when first creating the table as well.

### Does this PR introduce any user-facing change?

Introduces a new interface

### How was this patch tested?

Will add tests once interface is vetted.

Closes #26913 from brkyvz/catalogOptions.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2020-01-09 11:18:16 -08:00
Gengliang Wang 94fc0e3235 [SPARK-30428][SQL] File source V2: support partition pruning
### What changes were proposed in this pull request?

File source V2: support partition pruning.
Note: subquery predicates are not pushed down for partition pruning even after this PR, due to the limitation for the current data source V2 API and framework. The rule `PlanSubqueries` requires the subquery expression to be in the children or class parameters in `SparkPlan`, while the condition is not satisfied for `BatchScanExec`.

### Why are the changes needed?

It's important for reading performance.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

New unit tests for all the V2 file sources

Closes #27112 from gengliangwang/PartitionPruningInFileScan.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-09 21:53:37 +08:00
Gengliang Wang 0d589f410b [SPARK-30267][SQL][FOLLOWUP] Use while loop in Avro Array Deserializer
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/26907
It changes the for loop `for (element <- array.asScala)` to while loop

### Why are the changes needed?

As per https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex, we should use while loop for the performance-sensitive code.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests.

Closes #27127 from gengliangwang/SPARK-30267-FollowUp.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-07 22:39:25 -08:00
Ximo Guanter 604d6799df [SPARK-30226][SQL] Remove withXXX functions in WriteBuilder
### What changes were proposed in this pull request?
Adding a `LogicalWriteInfo` interface as suggested by cloud-fan in https://github.com/apache/spark/pull/25990#issuecomment-555132991

### Why are the changes needed?
It provides compile-time guarantees where we previously had none, which will make it harder to introduce bugs in the future.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Compiles and passes tests

Closes #26678 from edrevo/add-logical-write-info.

Lead-authored-by: Ximo Guanter <joaquin.guantergonzalbez@telefonica.com>
Co-authored-by: Ximo Guanter
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-06 23:53:45 +08:00
Steven Aerts e64512558f [SPARK-30267][SQL] Avro arrays can be of any List
The Deserializer assumed that avro arrays are always of type `GenericData$Array` which is not the case.
Assuming they are from java.util.List is safer and fixes a ClassCastException in some avro code.

### What changes were proposed in this pull request?
Java.util.List has all the necessary methods and is the base class of GenericData$Array.

### Why are the changes needed?
To prevent the following exception in more complex avro objects:

```
java.lang.ClassCastException: java.util.ArrayList cannot be cast to org.apache.avro.generic.GenericData$Array
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$newWriter$19(AvroDeserializer.scala:170)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$newWriter$19$adapted(AvroDeserializer.scala:169)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$getRecordWriter$1(AvroDeserializer.scala:314)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$getRecordWriter$1$adapted(AvroDeserializer.scala:310)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$getRecordWriter$2(AvroDeserializer.scala:332)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$getRecordWriter$2$adapted(AvroDeserializer.scala:329)
	at org.apache.spark.sql.avro.AvroDeserializer.$anonfun$converter$3(AvroDeserializer.scala:56)
	at org.apache.spark.sql.avro.AvroDeserializer.deserialize(AvroDeserializer.scala:70)
```

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
The current tests already test this behavior.  In essesence this patch just changes a type case to a more basic type.  So I expect no functional impact.

Closes #26907 from steven-aerts/spark-30267.

Authored-by: Steven Aerts <steven.aerts@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-03 12:44:49 -08:00
Jungtaek Lim (HeartSaVioR) 319ccd5711 [SPARK-30336][SQL][SS] Move Kafka consumer-related classes to its own package
### What changes were proposed in this pull request?

There're too many classes placed in a single package "org.apache.spark.sql.kafka010" which classes can be grouped by purpose.

As a part of change in SPARK-21869 (#26845), we moved out producer related classes to "org.apache.spark.sql.kafka010.producer" and only expose necessary classes/methods to the outside of package. This patch applies the same to consumer related classes.

### Why are the changes needed?

Described above.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing UTs.

Closes #26991 from HeartSaVioR/SPARK-30336.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-31 09:30:55 -06:00
Gengliang Wang 07593d362f [SPARK-27506][SQL][FOLLOWUP] Use option avroSchema to specify an evolved schema in from_avro
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/26780
In https://github.com/apache/spark/pull/26780, a new Avro data source option `actualSchema` is introduced for setting the original Avro schema in function `from_avro`, while the expected schema is supposed to be set in the parameter `jsonFormatSchema` of `from_avro`.

However, there is another Avro data source option `avroSchema`. It is used for setting the expected schema in readiong and writing.

This PR is to use the option `avroSchema` option for  reading Avro data with an evolved schema and remove the new one `actualSchema`

### Why are the changes needed?

Unify and simplify the Avro data source options.

### Does this PR introduce any user-facing change?

Yes.
To deserialize Avro data with an evolved schema, before changes:
```
from_avro('col, expectedSchema, ("actualSchema" -> actualSchema))
```

After changes:
```
from_avro('col, actualSchema, ("avroSchema" -> expectedSchema))
```

The second parameter is always the actual Avro schema after changes.
### How was this patch tested?

Update the existing tests in https://github.com/apache/spark/pull/26780

Closes #27045 from gengliangwang/renameAvroOption.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-30 18:14:21 +09:00
Jungtaek Lim (HeartSaVioR) 481fb63f97 [MINOR][SQL][SS] Remove TODO comments as var in case class is discouraged but worth breaking it
### What changes were proposed in this pull request?

This patch removes TODO comments which are left to address changing case classes having vars to normal classes in spark-sql-kafka module - the pattern is actually discouraged, but still worth to break it, as we already use automatic toString implementation and we may be using more.

### Why are the changes needed?

Described above.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing UTs.

Closes #26992 from HeartSaVioR/SPARK-30337.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-26 11:47:41 +09:00
Jungtaek Lim (HeartSaVioR) 2164243526 [SPARK-28144][SPARK-29294][SS][FOLLOWUP] Use SystemTime defined in Kafka Time interface
### What changes were proposed in this pull request?

This patch addresses review comments in #26960 (https://github.com/apache/spark/pull/26960#discussion_r360661930 / https://github.com/apache/spark/pull/26960#discussion_r360661947) which were not addressed in the patch. Addressing these review comments will let the code less dependent on actual implementation as it only relies on `Time` interface in Kafka.

### Why are the changes needed?

These were review comments in previous PR and they bring actual benefit though they're minors.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #26979 from HeartSaVioR/SPARK-29294-follow-up.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-24 11:39:03 +09:00
Jungtaek Lim (HeartSaVioR) 7bff2db9ed [SPARK-21869][SS] Revise Kafka producer pool to implement 'expire' correctly
This patch revises Kafka producer pool (cache) to implement 'expire' correctly.

Current implementation of Kafka producer cache leverages Guava cache, which decides cached producer instance to be expired if the instance is not "accessed" from cache. The behavior defines expiration time as "last accessed time + timeout", which is incorrect because some task may use the instance longer than timeout. There's no concept of "returning" in Guava cache as well, so it cannot be fixed with Guava cache.

This patch introduces a new pool implementation which tracks "reference count" of cached instance, and defines expiration time for the instance as "last returned time + timeout" if the reference count goes 0, otherwise Long.MaxValue (effectively no expire). Expiring instances will be done with evict thread explicitly instead of evicting in part of handling acquire. (It might bring more overhead, but it ensures clearing expired instances even the pool is idle.)

This patch also creates a new package `producer` under `kafka010`, to hide the details from `kafka010` package. In point of `kafka010` package's view, only acquire()/release()/reset() are available in pool, and even for CachedKafkaProducer the package cannot close the producer directly.

Explained above.

Yes, but only for the way of expiring cached instances. (The difference is described above.) Each executor leveraging spark-sql-kafka would have one eviction thread.

New and existing UTs.

Closes #26845 from HeartSaVioR/SPARK-21869-revised.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-23 14:19:33 -08:00
Kazuaki Ishizaki f31d9a629b [MINOR][DOC][SQL][CORE] Fix typo in document and comments
### What changes were proposed in this pull request?

Fixed typo in `docs` directory and in other directories

1. Find typo in `docs` and apply fixes to files in all directories
2. Fix `the the` -> `the`

### Why are the changes needed?

Better readability of documents

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

No test needed

Closes #26976 from kiszk/typo_20191221.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 14:08:58 -08:00
Jungtaek Lim (HeartSaVioR) 8384ff4c9d [SPARK-28144][SPARK-29294][SS] Upgrade Kafka to 2.4.0
### What changes were proposed in this pull request?

This patch upgrades the version of Kafka to 2.4, which supports Scala 2.13.

There're some incompatible changes in Kafka 2.4 which the patch addresses as well:

* `ZkUtils` is removed -> Replaced with `KafkaZkClient`
* Majority of methods are removed in `AdminUtils` -> Replaced with `AdminZkClient`
* Method signature of `Scheduler.schedule` is changed (return type) -> leverage `DeterministicScheduler` to avoid implementing `ScheduledFuture`

### Why are the changes needed?

* Kafka 2.4 supports Scala 2.13

### Does this PR introduce any user-facing change?

No, as Kafka API is known to be compatible across versions.

### How was this patch tested?

Existing UTs

Closes #26960 from HeartSaVioR/SPARK-29294.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 14:01:25 -08:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

### Does this PR introduce any user-facing change?

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
Maxim Gekk b03ce63c05 [SPARK-30258][TESTS] Eliminate warnings of deprecated Spark APIs in tests
### What changes were proposed in this pull request?
In the PR, I propose to move all tests that use deprecated Spark APIs to separate test classes, and add the annotation:
```scala
deprecated("This test suite will be removed.", "3.0.0")
```
The annotation suppress warnings from already deprecated methods and classes.

### Why are the changes needed?
The warnings about deprecated Spark APIs in tests does not indicate any issues because the tests use such APIs intentionally. Eliminating the warnings allows to highlight other warnings that could show real problems.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
By existing test suites and by
- DeprecatedAvroFunctionsSuite
- DeprecatedDateFunctionsSuite
- DeprecatedDatasetAggregatorSuite
- DeprecatedStreamingAggregationSuite
- DeprecatedWholeStageCodegenSuite

Closes #26885 from MaxGekk/eliminate-deprecate-warnings.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-16 18:24:32 -06:00
Jungtaek Lim (HeartSaVioR) 94eb66593a [SPARK-30227][SQL] Add close() on DataWriter interface
### What changes were proposed in this pull request?

This patch adds close() method to the DataWriter interface, which will become the place to cleanup the resource.

### Why are the changes needed?

The lifecycle of DataWriter instance ends at either commit() or abort(). That makes datasource implementors to feel they can place resource cleanup in both sides, but abort() can be called when commit() fails; so they have to ensure they don't do double-cleanup if cleanup is not idempotent.

### Does this PR introduce any user-facing change?

Depends on the definition of user; if they're developers of custom DSv2 source, they have to add close() in their DataWriter implementations. It's OK to just add close() with empty content as they should have already dealt with resource cleanup in commit/abort, but they would love to migrate the resource cleanup logic to close() as it avoids double cleanup. If they're just end users using the provided DSv2 source (regardless of built-in/3rd party), no change.

### How was this patch tested?

Existing tests.

Closes #26855 from HeartSaVioR/SPARK-30227.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-13 16:12:41 +08:00
Fokko Driesprong 99ea324b6f [SPARK-27506][SQL] Allow deserialization of Avro data using compatible schemas
Follow up of https://github.com/apache/spark/pull/24405

### What changes were proposed in this pull request?
The current implementation of _from_avro_ and _AvroDataToCatalyst_ doesn't allow doing schema evolution since it requires the deserialization of an Avro record with the exact same schema with which it was serialized.

The proposed change is to add a new option `actualSchema` to allow passing the schema used to serialize the records. This allows using a different compatible schema for reading by passing both schemas to _GenericDatumReader_. If no writer's schema is provided, nothing changes from before.

### Why are the changes needed?
Consider the following example.

```
// schema ID: 1
val schema1 = """
{
    "type": "record",
    "name": "MySchema",
    "fields": [
        {"name": "col1", "type": "int"},
        {"name": "col2", "type": "string"}
     ]
}
"""

// schema ID: 2
val schema2 = """
{
    "type": "record",
    "name": "MySchema",
    "fields": [
        {"name": "col1", "type": "int"},
        {"name": "col2", "type": "string"},
        {"name": "col3", "type": "string", "default": ""}
     ]
}
"""
```

The two schemas are compatible - i.e. you can use `schema2` to deserialize events serialized with `schema1`, in which case there will be the field `col3` with the default value.

Now imagine that you have two dataframes (read from batch or streaming), one with Avro events from schema1 and the other with events from schema2. **We want to combine them into one dataframe** for storing or further processing.

With the current `from_avro` function we can only decode each of them with the corresponding schema:

```
scalaval df1 = ... // Avro events created with schema1
df1: org.apache.spark.sql.DataFrame = [eventBytes: binary]
scalaval decodedDf1 = df1.select(from_avro('eventBytes, schema1) as "decoded")
decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string>]

scalaval df2= ... // Avro events created with schema2
df2: org.apache.spark.sql.DataFrame = [eventBytes: binary]
scalaval decodedDf2 = df2.select(from_avro('eventBytes, schema2) as "decoded")
decodedDf2: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string, col3: string>]
```

but then `decodedDf1` and `decodedDf2` have different Spark schemas and we can't union them. Instead, with the proposed change we can decode `df1` in the following way:

```
scalaimport scala.collection.JavaConverters._
scalaval decodedDf1 = df1.select(from_avro(data = 'eventBytes, jsonFormatSchema = schema2, options = Map("actualSchema" -> schema1).asJava) as "decoded")
decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct<col1: int, col2: string, col3: string>]
```

so that both dataframes have the same schemas and can be merged.

### Does this PR introduce any user-facing change?
This PR allows users to pass a new configuration but it doesn't affect current code.

### How was this patch tested?
A new unit test was added.

Closes #26780 from Fokko/SPARK-27506.

Lead-authored-by: Fokko Driesprong <fokko@apache.org>
Co-authored-by: Gianluca Amori <gianluca.amori@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-12-11 01:26:29 -08:00
Shixiong Zhu cfd7ca9a06
Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer"
This reverts commit 3641c3dd69.
2019-12-10 13:38:38 -08:00
Jungtaek Lim (HeartSaVioR) 538b8d101c [SPARK-30159][SQL][FOLLOWUP] Fix lint-java via removing unnecessary imports
### What changes were proposed in this pull request?

This patch fixes the Java code style violations in SPARK-30159 (#26788) which are caught by lint-java (Github Action caught it and I can reproduce it locally). Looks like Jenkins build may have different policy on checking Java style check or less accurate.

### Why are the changes needed?

Java linter starts complaining.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

lint-java passed locally

This closes #26819

Closes #26818 from HeartSaVioR/SPARK-30159-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-09 08:57:20 -08:00
Gengliang Wang a717d219a6 [SPARK-30159][SQL][TESTS] Fix the method calls of QueryTest.checkAnswer
### What changes were proposed in this pull request?

Before this PR, the method `checkAnswer` in Object `QueryTest` returns an optional string. It doesn't throw exceptions when errors happen.
The actual exceptions are thrown in the trait `QueryTest`.

However, there are some test suites(`StreamSuite`, `SessionStateSuite`, `BinaryFileFormatSuite`, etc.) that use the no-op method `QueryTest.checkAnswer` and expect it to fail test cases when the execution results don't match the expected answers.

After this PR:
1. the method `checkAnswer` in Object `QueryTest` will fail tests on errors or unexpected results.
2. add a new method `getErrorMessageInCheckAnswer`, which is exactly the same as the previous version of `checkAnswer`. There are some test suites use this one to customize the test failure message.
3. for the test suites that extend the trait `QueryTest`, we should use the method `checkAnswer` directly, instead of calling the method from Object `QueryTest`.

### Why are the changes needed?

We should fix these method calls to perform actual validations in test suites.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing unit tests.

Closes #26788 from gengliangwang/fixCheckAnswer.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-09 22:19:08 +09:00
angerszhu da27f91560 [SPARK-29957][TEST] Reset MiniKDC's default enctypes to fit jdk8/jdk11
### What changes were proposed in this pull request?

Hadoop jira: https://issues.apache.org/jira/browse/HADOOP-12911
In this jira, the author said to replace origin Apache Directory project which is not maintained (but not said it won't work well in jdk11) to Apache Kerby which is java binding(fit java version).

And in Flink: https://github.com/apache/flink/pull/9622
Author show the reason why hadoop-2.7.2's  `MminiKdc` failed with jdk11.
Because new encryption types of `es128-cts-hmac-sha256-128` and `aes256-cts-hmac-sha384-192` (for Kerberos 5) enabled by default were added in Java 11.
Spark with `hadoop-2.7's MiniKdc`does not support these encryption types and does not work well when these encryption types are enabled, which results in the authentication failure.

And when I test hadoop-2.7.2's minikdc in local, the kerberos 's debug error message is  read message stream failed, message can't match.

### Why are the changes needed?
Support jdk11 with hadoop-2.7

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
Existed UT

Closes #26594 from AngersZhuuuu/minikdc-3.2.0.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-05 23:12:45 -08:00
Ximo Guanter 54c5087a3a [SPARK-29248][SQL] provider number of partitions when creating v2 data writer factory
### What changes were proposed in this pull request?
When implementing a ScanBuilder, we require the implementor to provide the schema of the data and the number of partitions.

However, when someone is implementing WriteBuilder we only pass them the schema, but not the number of partitions. This is an asymetrical developer experience.

This PR adds a PhysicalWriteInfo interface that is passed to createBatchWriterFactory and createStreamingWriterFactory that adds the number of partitions of the data that is going to be written.

### Why are the changes needed?
Passing in the number of partitions on the WriteBuilder would enable data sources to provision their write targets before starting to write. For example:

it could be used to provision a Kafka topic with a specific number of partitions
it could be used to scale a microservice prior to sending the data to it
it could be used to create a DsV2 that sends the data to another spark cluster (currently not possible since the reader wouldn't be able to know the number of partitions)
### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Tests passed

Closes #26591 from edrevo/temp.

Authored-by: Ximo Guanter <joaquin.guantergonzalbez@telefonica.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-22 00:19:25 +08:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

### Does this PR introduce any user-facing change?

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06:00
shivsood a834dba120 Revert "[SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils
This reverts commit f7e53865 i.e PR #26301 from master

Closes #26583 from shivsood/revert_29644_master.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:44:16 -08:00
shivsood 32d44b1d0e [SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils
### What changes were proposed in this pull request?
Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType  as setShort() and setByte(). Changes in JDBCUtils.scala
Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType.

#### Problems

- In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively.
```
case ShortType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
stmt.setInt(pos + 1, row.getShort(pos))
The issue was pointed out by maropu

case ByteType =>
(stmt: PreparedStatement, row: Row, pos: Int) =>
 stmt.setInt(pos + 1, row.getByte(pos))
```

- Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType()

``` case java.sql.Types.TINYINT       => IntegerType ```

- At line 172 ShortType was wrongly interpreted as IntegerType
``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ```

- All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes.

### Why are the changes needed?
A given type should be set using the right type.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Corrected Unit test cases where applicable. Validated in CI/CD
Added a test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows.
```
./build/mvn install -DskipTests
./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12
```

Closes #26301 from shivsood/shorttype_fix_maropu.

Authored-by: shivsood <shivsood@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-13 17:56:13 -08:00
Gabor Somogyi 3641c3dd69 [SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer
### What changes were proposed in this pull request?

Kafka producers are now closed when `spark.kafka.producer.cache.timeout` reached which could be significant problem when processing big SQL queries. The workaround was to increase `spark.kafka.producer.cache.timeout` to a number where the biggest SQL query can be finished.

In this PR I've adapted similar solution which already exists on the consumer side, namely applies Apache Commons Pool on the producer side as well. Main advantages choosing this solution:
* Producers are not closed until they're in use
* No manual reference counting needed (which may be error prone)
* Thread-safe by design
* Provides jmx connection to the pool where metrics can be fetched

What this PR contains:
* Introduced producer side parameters to configure pool
* Renamed `InternalKafkaConsumerPool` to `InternalKafkaConnectorPool` and made it abstract
* Created 2 implementations from it: `InternalKafkaConsumerPool` and `InternalKafkaProducerPool`
* Adapted `CachedKafkaProducer` to use `InternalKafkaProducerPool`
* Changed `KafkaDataWriter` and `KafkaDataWriteTask` to release producer even in failure scenario
* Added several new tests
* Extended `KafkaTest` to clear not only producers but consumers as well
* Renamed `InternalKafkaConsumerPoolSuite` to `InternalKafkaConnectorPoolSuite` where only consumer tests are checking the behavior (please see comment for reasoning)

What this PR not yet contains(but intended when the main concept is stable):
* User facing documentation

### Why are the changes needed?
Kafka producer closed after 10 minutes (with default settings).

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existing + additional unit tests.
Cluster tests being started.

Closes #25853 from gaborgsomogyi/SPARK-21869.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-07 17:06:32 -08:00
Kent Yao 9562b26914 [SPARK-29757][SQL] Move calendar interval constants together
### What changes were proposed in this pull request?
```java
  public static final int YEARS_PER_DECADE = 10;
  public static final int YEARS_PER_CENTURY = 100;
  public static final int YEARS_PER_MILLENNIUM = 1000;

  public static final byte MONTHS_PER_QUARTER = 3;
  public static final int MONTHS_PER_YEAR = 12;

  public static final byte DAYS_PER_WEEK = 7;
  public static final long DAYS_PER_MONTH = 30L;

  public static final long HOURS_PER_DAY = 24L;

  public static final long MINUTES_PER_HOUR = 60L;

  public static final long SECONDS_PER_MINUTE = 60L;
  public static final long SECONDS_PER_HOUR = MINUTES_PER_HOUR * SECONDS_PER_MINUTE;
  public static final long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR;

  public static final long MILLIS_PER_SECOND = 1000L;
  public static final long MILLIS_PER_MINUTE = SECONDS_PER_MINUTE * MILLIS_PER_SECOND;
  public static final long MILLIS_PER_HOUR = MINUTES_PER_HOUR * MILLIS_PER_MINUTE;
  public static final long MILLIS_PER_DAY = HOURS_PER_DAY * MILLIS_PER_HOUR;

  public static final long MICROS_PER_MILLIS = 1000L;
  public static final long MICROS_PER_SECOND = MILLIS_PER_SECOND * MICROS_PER_MILLIS;
  public static final long MICROS_PER_MINUTE = SECONDS_PER_MINUTE * MICROS_PER_SECOND;
  public static final long MICROS_PER_HOUR = MINUTES_PER_HOUR * MICROS_PER_MINUTE;
  public static final long MICROS_PER_DAY = HOURS_PER_DAY * MICROS_PER_HOUR;
  public static final long MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY;
  /* 365.25 days per year assumes leap year every four years */
  public static final long MICROS_PER_YEAR = (36525L * MICROS_PER_DAY) / 100;

  public static final long NANOS_PER_MICROS = 1000L;
  public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
  public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
```
The above parameters are defined in IntervalUtils, DateTimeUtils, and CalendarInterval, some of them are redundant, some of them are cross-referenced.

### Why are the changes needed?
To simplify code, enhance consistency and reduce risks

### Does this PR introduce any user-facing change?

no
### How was this patch tested?

modified uts

Closes #26399 from yaooqinn/SPARK-29757.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 19:48:19 +08:00
Jungtaek Lim (HeartSaVioR) 252ecd333f [SPARK-29635][SS] Extract base test suites between Kafka micro-batch sink and Kafka continuous sink
### What changes were proposed in this pull request?

This patch leverages V2 continuous memory stream to extract tests from Kafka micro-batch sink suite and continuous sink suite and deduplicate them. These tests are basically doing the same, except how to run and verify the result.

### Why are the changes needed?

We no longer have same tests spotted on two places - brings 300 lines deletion.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing UTs.

Closes #26292 from HeartSaVioR/SPARK-29635.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-06 17:08:42 -08:00
Sean Owen 19b8c71436 [SPARK-29674][CORE] Update dropwizard metrics to 4.1.x for JDK 9+
### What changes were proposed in this pull request?

Update the version of dropwizard metrics that Spark uses for metrics to 4.1.x, from 3.2.x.

### Why are the changes needed?

This helps JDK 9+ support, per for example https://github.com/dropwizard/metrics/pull/1236

### Does this PR introduce any user-facing change?

No, although downstream users with custom metrics may be affected.

### How was this patch tested?

Existing tests.

Closes #26332 from srowen/SPARK-29674.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 15:13:06 -08:00
dengziming 28ccd31aee [SPARK-29611][WEBUI] Sort Kafka metadata by the number of messages
### What changes were proposed in this pull request?

Sort metadata by the number of messages in each Kafka partition

### Why are the changes needed?

help to find the data skewness problem.

### Does this PR introduce any user-facing change?

Yes, add a column count to the metadata and sort by count
![image](https://user-images.githubusercontent.com/26023240/67617886-63e06800-f81a-11e9-8718-be3a0100952e.png)

If you set `minPartitions` configurations with structure structured-streaming which doesn't have the Streaming page, my code changes in `DirectKafkaInputDStream` won't affect the WEB UI page just as it shows in the follow image

![image](https://user-images.githubusercontent.com/26023240/68020762-79520800-fcda-11e9-96cd-f0c64a36f505.png)

### How was this patch tested?

Manual test

Closes #26266 from dengziming/feature_ui_optimize.

Lead-authored-by: dengziming <dengziming@growingio.com>
Co-authored-by: dengziming <swzmdeng@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-01 22:46:34 -07:00
Xingbo Jiang 8207c835b4 Revert "Prepare Spark release v3.0.0-preview-rc2"
This reverts commit 007c873ae3.
2019-10-30 17:45:44 -07:00
Xingbo Jiang 007c873ae3 Prepare Spark release v3.0.0-preview-rc2
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

N/A
2019-10-30 17:42:59 -07:00
Xingbo Jiang b33a58c0c6 Revert "Prepare Spark release v3.0.0-preview-rc1"
This reverts commit 5eddbb5f1d.
2019-10-28 22:32:34 -07:00
Xingbo Jiang 5eddbb5f1d Prepare Spark release v3.0.0-preview-rc1
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

N/A

Closes #26243 from jiangxb1987/3.0.0-preview-prepare.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-28 22:31:29 -07:00
Jungtaek Lim (HeartSaVioR) 762db39c15 [SPARK-29509][SQL][SS] Deduplicate codes from Kafka data source
### What changes were proposed in this pull request?

This patch deduplicates code blocks in Kafka data source which are being repeated multiple times in a method.

### Why are the changes needed?

This change would simplify the code and open possibility to simplify future code whenever fields are added to Kafka writer schema.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing UTs.

Closes #26158 from HeartSaVioR/MINOR-deduplicate-kafka-source.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-28 11:14:18 -07:00
Gabor Somogyi 25493919f8 [SPARK-29580][TESTS] Add kerberos debug messages for Kafka secure tests
### What changes were proposed in this pull request?
`org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite` failed lately. After had a look at the logs it just shows the following fact without any details:
```
Caused by: sbt.ForkMain$ForkError: sun.security.krb5.KrbException: Server not found in Kerberos database (7) - Server not found in Kerberos database
```
Since the issue is intermittent and not able to reproduce it we should add more debug information and wait for reproduction with the extended logs.

### Why are the changes needed?
Failing test doesn't give enough debug information.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
I've started the test manually and checked that such additional debug messages show up:
```
>>> KrbApReq: APOptions are 00000000 00000000 00000000 00000000
>>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType
Looking for keys for: kafka/localhostEXAMPLE.COM
Added key: 17version: 0
Added key: 23version: 0
Added key: 16version: 0
Found unsupported keytype (3) for kafka/localhostEXAMPLE.COM
>>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType
Using builtin default etypes for permitted_enctypes
default etypes for permitted_enctypes: 17 16 23.
>>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType
MemoryCache: add 1571936500/174770/16C565221B70AAB2BEFE31A83D13A2F4/client/localhostEXAMPLE.COM to client/localhostEXAMPLE.COM|kafka/localhostEXAMPLE.COM
MemoryCache: Existing AuthList:
#3: 1571936493/200803/8CD70D280B0862C5DA1FF901ECAD39FE/client/localhostEXAMPLE.COM
#2: 1571936499/985009/BAD33290D079DD4E3579A8686EC326B7/client/localhostEXAMPLE.COM
#1: 1571936499/995208/B76B9D78A9BE283AC78340157107FD40/client/localhostEXAMPLE.COM
```

Closes #26252 from gaborgsomogyi/SPARK-29580.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-25 14:11:35 -07:00