Commit graph

9387 commits

Author SHA1 Message Date
iRakson a40a2f8338 [SPARK-27619][SQL][FOLLOWUP] Rename 'spark.sql.legacy.useHashOnMapType' to 'spark.sql.legacy.allowHashOnMapType'
### What changes were proposed in this pull request?
Renamed configuration from `spark.sql.legacy.useHashOnMapType` to `spark.sql.legacy.allowHashOnMapType`.

### Why are the changes needed?
Better readability of configuration.

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

### How was this patch tested?
Existing UTs.

Closes #27719 from iRakson/SPARK-27619_FOLLOWUP.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-28 22:57:50 +08:00
Eric Wu eba2076ca3 [SPARK-30842][SQL] Adjust abstraction structure for join operators
### What changes were proposed in this pull request?
Currently the join operators are not well abstracted, since there are lot of common logic. A trait can be created for easier pattern matching and other future handiness. This is a follow-up PR based on comment
https://github.com/apache/spark/pull/27509#discussion_r379613391 .

This PR refined from the following aspects:
1. Refined structure of all physical join operators
2. Add missing joinType field for CartesianProductExec operator
3. Refined codes related to Explain Formatted

The EXPLAIN FORMATTED changes are
1. Converge all join operator `verboseStringWithOperatorId` implementations to `BaseJoinExec`. Join condition displayed, and join keys displayed if it’s not empty.
2. `#1` will add Join condition to `BroadcastNestedLoopJoinExec`.
3. `#1` will **NOT** affect `CartesianProductExec`,`SortMergeJoin` and `HashJoin`s, since they already got there override implementation before.
4. Converge all join operator `simpleStringWithNodeId` to `BaseJoinExec`, which will enhance the one line description for `CartesianProductExec` with `JoinType` added.
5. Override `simpleStringWithNodeId` in `BroadcastNestedLoopJoinExec` to show `BuildSide`, which was only done for `HashJoin`s before.

### Why are the changes needed?
Make the code consistent with other operators and for future handiness of join operators.

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

### How was this patch tested?
Existing tests

Closes #27595 from Eric5553/RefineJoin.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-28 18:42:15 +08:00
Wenchen Fan f21894e5fa [SPARK-30902][SQL] Default table provider should be decided by catalog implementations
### What changes were proposed in this pull request?

When `CREATE TABLE` SQL statement does not specify the provider, leave it to the catalog implementations to decide.

### Why are the changes needed?

It's super weird if we set the default provider to parquet when creating a table in a JDBC catalog.

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

Yes, v2 catalog will not see a "provider" property in table properties if it's not specified in `CREATE TABLE` SQL statement. V2 catalog is new in 3.0.

### How was this patch tested?

new tests

Closes #27650 from cloud-fan/create_table.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-28 15:14:23 +09:00
yi.wu a1d2ce90b0 [SPARK-30972][SQL] PruneHiveTablePartitions should be executed as earlyScanPushDownRules
### What changes were proposed in this pull request?

Make rule `PruneHiveTablePartitions` to execute as `earlyScanPushDownRules`.

### Why are the changes needed?

Similar to rule `PruneFileSourcePartitions`, `PruneHiveTablePartitions` should also be executed as earlyScanPushDownRules to eliminate the impact on statistic computation later.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27723 from Ngone51/early_hive_prune.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-28 11:50:20 +08:00
Liang-Chi Hsieh ba032acf95 [SPARK-30955][SQL] Exclude Generate output when aliasing in nested column pruning
### What changes were proposed in this pull request?

When aliasing in nested column pruning in Project on top of Generate, we should exclude Generate outputs.

### Why are the changes needed?

Right now we would prune nested columns in Project on top of Generate. It is possible that referred nested columns are from Generate's outputs, not from its child. To address that case, we should exclude Generate outputs when aliasing in nested column pruning.

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

No.

### How was this patch tested?

Unit test.

Closes #27702 from viirya/fix-nested-pruning.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-28 12:29:46 +09:00
Takeshi Yamamuro 314442a3d5 [SQL][MINOR][TESTS] Remove GivenWhenThen trait from HiveComparisonTest
### What changes were proposed in this pull request?

This pr intends to remove non-used trait, `GivenWhenThen`, from `HiveComparisonTest`.

### Why are the changes needed?

For better code.

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

No

### How was this patch tested?

Existing tests.

Closes #27726 from maropu/MINOR-20200228.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-27 17:19:27 -08:00
Kent Yao 2d2706cb86 [SPARK-30956][SQL][TESTS] Use intercept instead of try-catch to assert failures in IntervalUtilsSuite
### What changes were proposed in this pull request?

In this PR, I addressed the comment from https://github.com/apache/spark/pull/27672#discussion_r383719562 to use `intercept` instead of `try-catch` block to assert  failures in the IntervalUtilsSuite

### Why are the changes needed?

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

no

### How was this patch tested?

Nah

Closes #27700 from yaooqinn/intervaltest.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-02-27 23:12:35 +09:00
yi.wu 22dfd15a45 [SPARK-30937][DOC] Group Hive upgrade guides together
### What changes were proposed in this pull request?

This PR groups all hive upgrade related migration guides inside Spark 3.0 together.

Also add another behavior change of `ScriptTransform` in the new Hive section.

### Why are the changes needed?

Make the doc more clearly to user.

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

No, new doc for Spark 3.0.

### How was this patch tested?

N/A.

Closes #27670 from Ngone51/hive_migration.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 21:29:42 +08:00
Eric Wu bce8d9354c [SPARK-30765][SQL] Refine base operator abstraction code style
### What changes were proposed in this pull request?
When doing base operator abstraction work, we found there are still some code snippet is  inconsistent with other abstraction code style. This PR addressed following two code refactor cases.

**Case 1** Override keyword missed for some fields in derived classes. The compiler will not capture it if we rename some fields in the future.

**Case 2** Inconsistent abstract class field definition. The updated style will simplify derived class definition, e.g. `EvalPythonExec` `WindowExecBase`

### Why are the changes needed?
Improve the code style consistency and code quality

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

### How was this patch tested?
Existing tests

Closes #27511 from Eric5553/BaseClassAbstraction.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 16:40:10 +09:00
Liang-Chi Hsieh 160c144baa [SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type
### What changes were proposed in this pull request?

This patch proposes to throw clear analysis exception if untyped `Dataset.select` takes typed column expression that needs input type.

### Why are the changes needed?

`Dataset` provides few typed `select` helper functions to select typed column expressions. The maximum number of typed columns supported is 5. If wanting to select more than 5 typed columns, it silently calls untyped `Dataset.select` and can causes weird unresolved error, like:

```
org.apache.spark.sql.AnalysisException: unresolved operator 'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141];;
'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141]
+- Project [_1#6 AS a#13, _2#7 AS b#14, _3#8 AS c#15, _4#9 AS d#16, _5#10 AS e#17, _6#11 AS F#18]
 +- LocalRelation [_1#6, _2#7, _3#8, _4#9, _5#10, _6#11]

at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:43)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:95)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:431)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:430)
 at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:430)
```

However, to fully disallow typed columns as input to untyped `select` API will break current usage like `count` that is a `TypedColumn` in `functions`. In order to keep compatibility, we should allow current usage of certain `TypedColumn`s as input to untyped `select` API. For the `TypedColumn`s that will cause unresolved exception, we should explicitly let users know that they are incorrectly calling untyped `select` with typed columns which need input type.

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

Yes, but this PR only refines the error message.

When users call `Dataset.select` API with typed column that needs input type, an analysis exception will be thrown. Previously an unresolved error will be thrown.

### How was this patch tested?

Unit tests.

Closes #27499 from viirya/SPARK-30590.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 14:09:07 +08:00
beliefer 1515d45b8d [SPARK-27924][SQL][FOLLOW-UP] Improve ANSI SQL Boolean-Predicate
### What changes were proposed in this pull request?
This PR follows https://github.com/apache/spark/pull/25074 and improves the implement.

### Why are the changes needed?
Improve code.

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

### How was this patch tested?
Exists UT

Closes #27699 from beliefer/improve-boolean-test.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 13:42:02 +08:00
HyukjinKwon cd3ef2249f [SPARK-30906][SQL][TESTS][FOLLOW-UP] Set the configuration against TestHive explicitly in HiveSerDeSuite
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/27659 (see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3/253/), the tests below fail consistently, specifically in one job https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3/ in Jenkins

```
org.apache.spark.sql.hive.execution.HiveSerDeSuite.Test the default fileformat for Hive-serde tables
```

The profile is same as PR builder but seems it fails specifically in this machine.

Several configurations used in `HiveSerDeSuite` are not being set presumably due to the inconsistency between `SQLConf.get` and the active Spark session described in the https://github.com/apache/spark/pull/27387, and as a side effect of the cloned session at https://github.com/apache/spark/pull/27659.

This PR proposes to explicitly set the configuration against `TestHive` by using `withExistingConf` at `withSQLConf`

### Why are the changes needed?

To make `spark-master-test-sbt-hadoop-2.7-hive-2.3` job pass.

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

No.

### How was this patch tested?

Cannot reproduce in my local. Presumably it cannot be reproduced in the PR builder. We should see if the tests pass at `spark-master-test-sbt-hadoop-2.7-hive-2.3` job after this PR is merged.

Closes #27705 from HyukjinKwon/SPARK-30906.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-26 18:01:26 -08:00
beliefer 825d3dc11b [SPARK-30841][SQL][DOC] Add version information to the configuration of SQL
### What changes were proposed in this pull request?
Add version information to the configuration of Spark SQL.
Note: Because SQLConf has a lot of configuration items, I split the items into two PR. Another PR will follows this PR.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.analyzer.maxIterations | 3.0.0 | SPARK-30138 | c2f29d5ea58eb4565cc5602937d6d0bb75558513#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.excludedRules | 2.4.0 | SPARK-24802 | 434319e73f8cb6e080671bdde42a72228bd814ef#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.maxIterations | 2.0.0 | SPARK-14677 | f4be0946af219379fb2476e6f80b2e50463adeb2#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.optimizer.inSetConversionThreshold | 2.0.0 | SPARK-14796 | 3647120a5a879edf3a96a5fd68fb7aa849ad57ef#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.optimizer.inSetSwitchThreshold | 3.0.0 | SPARK-26205 | 0c23a39384b7ae5fb4aeb4f7f6fe72007b84bbd2#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.planChangeLog.level | 3.0.0 | SPARK-25415 | 8b702e1e0aba1d3e4b0aa582f20cf99f80a44a09#diff-9a6b543db706f1a90f790783d6930a13 | This configuration does not exist in branch-2.4 branch, but from the branch-3.0 git log, it is found that the version number of the pom.xml file is 2.4.0-SNAPSHOT
spark.sql.optimizer.planChangeLog.rules | 3.0.0 | SPARK-25415 | 8b702e1e0aba1d3e4b0aa582f20cf99f80a44a09#diff-9a6b543db706f1a90f790783d6930a13 | This configuration does not exist in branch-2.4 branch, but from the branch-3.0 git log, it is found that the version number of the pom.xml file is 2.4.0-SNAPSHOT
spark.sql.optimizer.planChangeLog.batches | 3.0.0 | SPARK-27088 | 074533334d01afdd7862a1ac6c5a7a672bcce3f8#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.enabled | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.useStats | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly | 3.0.0 | SPARK-30528 | 59a13c9b7bc3b3aa5b5bc30a60344f849c0f8012#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.inMemoryColumnarStorage.compressed | 1.0.1 | SPARK-2631 | 86534d0f5255362618c05a07b0171ec35c915822#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryColumnarStorage.batchSize | 1.1.1 | SPARK-2650 | 779d1eb26d0f031791e93c908d51a59c3b422a55#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryColumnarStorage.partitionPruning | 1.2.0 | SPARK-2961 | 248067adbe90f93c7d5e23aa61b3072dfdf48a8a#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryTableScanStatistics.enable | 3.0.0 | SPARK-28257 | 42b80ae128ab1aa8a87c1376fe88e2cde52e6e4f#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.inMemoryColumnarStorage.enableVectorizedReader | 2.3.1 | SPARK-23312 | e5e9f9a430c827669ecfe9d5c13cc555fc89c980#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.columnVector.offheap.enabled | 2.3.0 | SPARK-20101 | 572af5027e45ca96e0d283a8bf7c84dcf476f9bc#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.join.preferSortMergeJoin | 2.0.0 | SPARK-13977 | 9c23c818ca0175c8f2a4a66eac261ec251d27c97#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.sort.enableRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.autoBroadcastJoinThreshold | 1.1.0 | SPARK-2393 | c7db274be79f448fda566208946cb50958ea9b1a#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.limit.scaleUpFactor | 2.1.1 | SPARK-19944 | 80ebca62cbdb7d5c8606e95a944164ab1a943694#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.hive.advancedPartitionPredicatePushdown.enabled | 2.3.0 | SPARK-20331 | d8cada8d1d3fce979a4bc1f9879593206722a3b9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.shuffle.partitions | 1.1.0 | SPARK-1508 | 08ed9ad81397b71206c4dc903bfb94b6105691ed#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.adaptive.enabled | 1.6.0 | SPARK-9858 and SPARK-9859 and SPARK-9861 | d728d5c98658c44ed2949b55d36edeaa46f8c980#diff-41ef65b9ef5b518f77e2a03559893f4d |
spark.sql.adaptive.forceApply | 3.0.0 | SPARK-30719 | b29cb1a82b1a1facf1dd040025db93d998dad4cd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.reducePostShufflePartitions | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.minNumPostShufflePartitions | 3.0.0 | SPARK-9853 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.targetPostShuffleInputSize | 1.6.0 | SPARK-9858 and SPARK-9859 and SPARK-9861 | d728d5c98658c44ed2949b55d36edeaa46f8c980#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.adaptive.shuffle.maxNumPostShufflePartitions | 3.0.0 | SPARK-9853 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.localShuffleReader.enabled | 3.0.0 | SPARK-29893 | 6e581cf164c3a2930966b270ac1406dc1195c942#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.skewedJoinOptimization.enabled | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.skewedJoinOptimization.skewedPartitionFactor | 3.0.0 | SPARK-30812 | 5b36cdbbfef147e93b35eaa4f8e0bea9690b6d06#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin | 3.0.0 | SPARK-9853 and SPARK-29002 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 and b2f06608b785f577999318c00f2c315f39d90889#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.subexpressionElimination.enabled | 1.6.0 | SPARK-10371 | f38509a763816f43a224653fe65e4645894c9fc4#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.caseSensitive | 1.4.0 | SPARK-4699 | 21bd7222e55b9cf684c072141998a0623a69f514#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.constraintPropagation.enabled | 2.2.0 | SPARK-19846 | e011004bedca47be998a0c14fe22a6f9bb5090cd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parser.escapedStringLiterals | 2.2.1 | SPARK-20399 | 3d1908fd58fd9b1970cbffebdb731bfe4c776ad9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.fileCompressionFactor | 2.3.1 | SPARK-22790 | 0fc5533e53ad03eb67590ddd231f40c2713150c3#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.mergeSchema | 1.5.0 | SPARK-8690 | 246265f2bb056d5e9011d3331b809471a24ff8d7#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.respectSummaryFiles | 1.5.0 | SPARK-8838 | 6175d6cfe795fbd88e3ee713fac375038a3993a8#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.binaryAsString | 1.1.1 | SPARK-2927 | de501e169f24e4573747aec85b7651c98633c028#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.int96AsTimestamp | 1.3.0 | SPARK-4987 | 67d52207b5cf2df37ca70daff2a160117510f55e#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.int96TimestampConversion | 2.3.0 | SPARK-12297 | acf7ef3154e094875fa89f30a78ab111b267db91#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.outputTimestampType | 2.3.0 | SPARK-10365 | 21a7bfd5c324e6c82152229f1394f26afeae771c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.compression.codec | 1.1.1 | SPARK-3131 | 3a9d874d7a46ab8b015631d91ba479d9a0ba827f#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.filterPushdown | 1.2.0 | SPARK-4391 | 576688aa2a19bd4ba239a2b93af7947f983e5124#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.filterPushdown.date | 2.4.0 | SPARK-23727 | b02e76cbffe9e589b7a4e60f91250ca12a4420b2#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.filterPushdown.timestamp | 2.4.0 | SPARK-24718 | 43e4e851b642bbee535d22e1b9e72ec6b99f6ed4#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.filterPushdown.decimal | 2.4.0 | SPARK-24549 | 9549a2814951f9ba969955d78ac4bd2240f85989#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.filterPushdown.string.startsWith | 2.4.0 | SPARK-24638 | 03545ce6de08bd0ad685c5f59b73bc22dfc40887#diff-9a6b543db706f1a90f790783d6930a13 | 
spark.sql.parquet.pushdown.inFilterThreshold | 2.4.0 | SPARK-17091 | e1de34113e057707dfc5ff54a8109b3ec7c16dfb#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.writeLegacyFormat | 1.6.0 | SPARK-10400 | 01cd688f5245cbb752863100b399b525b31c3510#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.output.committer.class | 1.5.0 | SPARK-8139 | 111d6b9b8a584b962b6ae80c7aa8c45845ce0099#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.enableVectorizedReader | 2.0.0 | SPARK-13486 | 2b2c8c33236677c916541f956f7b94bba014a9ce#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.parquet.recordLevelFilter.enabled | 2.3.0 | SPARK-17310 | 673c67046598d33b9ecf864024ca7a937c1998d6#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.columnarReaderBatchSize | 2.4.0 | SPARK-23188 | cc41245fa3f954f961541bf4b4275c28473042b8#diff-9a6b543db706f1a90f790783d6930a13 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27691 from beliefer/add-version-to-sql-config-part-one.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 10:58:44 +09:00
Wenchen Fan eced93215f [SPARK-30918][SQL][FOLLOWUP] Fix typo in OptimizeSkewedJoin
### What changes were proposed in this pull request?

This is a follow up of #27669 in order to fix a typo.

### Why are the changes needed?

N/A

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

no

### How was this patch tested?

N/A

Closes #27714 from cloud-fan/typo.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-26 13:59:43 -08:00
iRakson c913b9d8b5 [SPARK-27619][SQL] MapType should be prohibited in hash expressions
### What changes were proposed in this pull request?
`hash()` and `xxhash64()` cannot be used on elements of `Maptype`. A new configuration `spark.sql.legacy.useHashOnMapType` is introduced to allow users to restore the previous behaviour.

When `spark.sql.legacy.useHashOnMapType` is set to false:

```
scala> spark.sql("select hash(map())");
org.apache.spark.sql.AnalysisException: cannot resolve 'hash(map())' due to data type mismatch: input to function hash cannot contain elements of MapType; line 1 pos 7;
'Project [unresolvedalias(hash(map(), 42), None)]
+- OneRowRelation
```

when `spark.sql.legacy.useHashOnMapType` is set to true :

```
scala> spark.sql("set spark.sql.legacy.useHashOnMapType=true");
res3: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("select hash(map())").first()
res4: org.apache.spark.sql.Row = [42]

```

### Why are the changes needed?

As discussed in Jira, SparkSql's map hashcodes depends on their order of insertion which is not consistent with the normal scala behaviour which might confuse users.
Code snippet from JIRA :
```
val a = spark.createDataset(Map(1->1, 2->2) :: Nil)
val b = spark.createDataset(Map(2->2, 1->1) :: Nil)

// Demonstration of how Scala Map equality is unaffected by insertion order:
assert(Map(1->1, 2->2).hashCode() == Map(2->2, 1->1).hashCode())
assert(Map(1->1, 2->2) == Map(2->2, 1->1))
assert(a.first() == b.first())

// In contrast, this will print two different hashcodes:
println(Seq(a, b).map(_.selectExpr("hash(*)").first()))
```

Also `MapType` is prohibited for aggregation / joins / equality comparisons #7819 and set operations #17236.

### Does this PR introduce any user-facing change?
Yes. Now users cannot use hash functions on elements of `mapType`. To restore the previous behaviour set `spark.sql.legacy.useHashOnMapType` to true.

### How was this patch tested?
UT added.

Closes #27580 from iRakson/SPARK-27619.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 01:48:12 +08:00
Terry Kim 73305475c1 [SPARK-30782][SQL] Column resolution doesn't respect current catalog/namespace for v2 tables
### What changes were proposed in this pull request?

This PR proposes to fix an issue where qualified columns are not matched for v2 tables if current catalog/namespace are used.

For v1 tables, you can currently perform the following:
```SQL
SELECT default.t.id FROM t;
```

For v2 tables, the following fails:
```SQL
USE testcat.ns1.ns2;
SELECT testcat.ns1.ns2.t.id FROM t;

org.apache.spark.sql.AnalysisException: cannot resolve '`testcat.ns1.ns2.t.id`' given input columns: [t.id, t.point]; line 1 pos 7;
```

### Why are the changes needed?

It is a bug since qualified column names cannot match if current catalog/namespace are used.

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

Yes, now the following works:
```SQL
USE testcat.ns1.ns2;
SELECT testcat.ns1.ns2.t.id FROM t;
```

### How was this patch tested?

Added new tests

Closes #27532 from imback82/qualifed_col_respect_current.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 00:21:38 +08:00
HyukjinKwon 020b2622e5 [SPARK-30798][SQL][TESTS][FOLLOW-UP] Set the configuration against the current session explicitly in HiveShowCreateTableSuite
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/27387 (see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3/202/), the tests below fail consistently, specifically in one job https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3/ in Jenkins

```
org.apache.spark.sql.hive.HiveShowCreateTableSuite.simple hive table
org.apache.spark.sql.hive.HiveShowCreateTableSuite.simple external hive table
org.apache.spark.sql.hive.HiveShowCreateTableSuite.hive bucketing is supported
```

The profile is same as PR builder but seems it fails specifically in this machine. Seems the legacy configuration `spark.sql.legacy.createHiveTableByDefault.enabled` is not being set due to the inconsistency between `SQLConf.get` and the active Spark session as described in the https://github.com/apache/spark/pull/27387.

This PR proposes to explicitly set the configuration against the session used instead of `SQLConf.get`.

### Why are the changes needed?

To make `spark-master-test-sbt-hadoop-2.7-hive-2.3` job pass.

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

No.

### How was this patch tested?

Cannot reproduce in my local. Presumably it cannot be reproduced in the PR builder. We should see if the tests pass at `spark-master-test-sbt-hadoop-2.7-hive-2.3` job after this PR is merged

Closes #27703 from HyukjinKwon/SPARK-30798-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 20:48:43 +09:00
gatorsmile 28b8713036 [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT
### What changes were proposed in this pull request?
This patch is to bump the master branch version to 3.1.0-SNAPSHOT.

### Why are the changes needed?
N/A

### Does this PR introduce any user-facing change?
N/A

### How was this patch tested?
N/A

Closes #27698 from gatorsmile/updateVersion.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-25 19:44:31 -08:00
Jungtaek Lim (HeartSaVioR) 9ea6c0a897
[SPARK-30943][SS] Show "batch ID" in tool tip string for Structured Streaming UI graphs
### What changes were proposed in this pull request?

This patch changes the tool tip string in Structured Streaming UI graphs to show batch ID (and timestamp as well) instead of only showing timestamp, which was a key for DStream but no longer a key for Structured Streaming.

This patch does some refactoring as there're some spots on confusion between js file for streaming and structured streaming.

Note that this patch doesn't actually change the x axis, as once we change it we should decouple the logic for graphs between streaming and structured streaming. It won't change UX meaningfully as in x axis we only show min and max which we still would like to know about "time" as well as batch ID.

### Why are the changes needed?

In Structured Streaming, everything is aligned for "batch ID" where the UI is only showing timestamp - end users have to manually find and correlate batch ID and the timestamp which is clearly a huge pain.

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

No

### How was this patch tested?

Manually tested. Screenshots:

![Screen Shot 2020-02-25 at 7 22 38 AM](https://user-images.githubusercontent.com/1317309/75197701-40b2ce80-57a2-11ea-9578-c2eb2d1091de.png)
![Screen Shot 2020-02-25 at 7 22 44 AM](https://user-images.githubusercontent.com/1317309/75197704-427c9200-57a2-11ea-9439-e0a8303d0860.png)
![Screen Shot 2020-02-25 at 7 22 58 AM](https://user-images.githubusercontent.com/1317309/75197706-43152880-57a2-11ea-9617-1276c3ba181e.png)
![Screen Shot 2020-02-25 at 7 23 04 AM](https://user-images.githubusercontent.com/1317309/75197708-43152880-57a2-11ea-9de2-7d37eaf88102.png)
![Screen Shot 2020-02-25 at 7 23 31 AM](https://user-images.githubusercontent.com/1317309/75197710-43adbf00-57a2-11ea-9ae4-4e292de39c36.png)

Closes #27687 from HeartSaVioR/SPARK-30943.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-25 15:29:36 -08:00
Wenchen Fan 8f247e5d36 [SPARK-30918][SQL] improve the splitting of skewed partitions
### What changes were proposed in this pull request?

Use the average size of the non-skewed partitions as the target size when splitting skewed partitions, instead of ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD

### Why are the changes needed?

The goal of skew join optimization is to make the data distribution move even. So it makes more sense the use the average size of the non-skewed partitions as the target size.

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

no

### How was this patch tested?

existing tests

Closes #27669 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-25 14:10:29 -08:00
Maxim Gekk ffc0935e64 [SPARK-30869][SQL] Convert dates to/from timestamps in microseconds precision
### What changes were proposed in this pull request?
In the PR, I propose to replace:

1. `millisToDays()` by `microsToDays()` which accepts microseconds since the epoch and returns days since the epoch in the specified time zone. The last one is the internal representation of Catalyst's DateType.
2. `daysToMillis()` by `daysToMicros()` which accepts days since the epoch in some time zone and returns the number of microseconds since the epoch. The last one is internal representation of Catalyst's TimestampType.
3. `fromMillis()` by `millisToMicros()`
4. `toMillis()` by `microsToMillis()`

### Why are the changes needed?
Spark stores timestamps in microseconds precision, so, there is no actual need to convert dates to milliseconds, and then to microseconds. As examples, look at DateTimeUtils functions `monthsBetween()` and `truncTimestamp()`.

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

### How was this patch tested?
By existing test suites UnivocityParserSuite, DateExpressionsSuite, ComputeCurrentTimeSuite, DateTimeUtilsSuite, DateFunctionsSuite, JsonSuite, StreamSuite.

Closes #27618 from MaxGekk/replace-millis-by-micros.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 23:05:28 +08:00
Kent Yao 761209c1f2 [SPARK-30919][SQL] Make interval multiply and divide's overflow behavior consistent with other operations
### What changes were proposed in this pull request?

The current behavior of interval multiply and divide follows the ANSI SQL standard when overflow, it is compatible with other operations when `spark.sql.ansi.enabled` is true, but not compatible when `spark.sql.ansi.enabled` is false.

When `spark.sql.ansi.enabled` is false, as the factor is a double value, so it should use java's rounding or truncation behavior for casting double to integrals. when divided by zero, it returns `null`.  we also follow the natural rules for intervals as defined in the Gregorian calendar, so we do not add the month fraction to days but add days fraction to microseconds.

### Why are the changes needed?

Make interval multiply and divide's overflow behavior consistent with other interval operations

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

no, these are new features in 3.0

### How was this patch tested?

add uts

Closes #27672 from yaooqinn/SPARK-30919.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 22:19:24 +08:00
Yuanjian Li e45f2c7fc0 [SPARK-28228][SQL][TESTS] Refactoring for nested CTE tests
### What changes were proposed in this pull request?
Split the nested CTE cases into a single file `cte-nested.sql`, which will be reused in cte-legacy.sql and cte-nonlegacy.sql.

### Why are the changes needed?
Make the cases easy to maintain.

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

### How was this patch tested?
Existing UT.

Closes #27667 from xuanyuanking/SPARK-28228-test.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-25 17:37:34 +09:00
Josh Rosen f152d2a0a8 [SPARK-30944][BUILD] Update URL for Google Cloud Storage mirror of Maven Central
### What changes were proposed in this pull request?

This PR is a followup to #27307: per https://travis-ci.community/t/maven-builds-that-use-the-gcs-maven-central-mirror-should-update-their-paths/5926, the Google Cloud Storage mirror of Maven Central has updated its URLs: the new paths are updated more frequently. The new paths are listed on https://storage-download.googleapis.com/maven-central/index.html

This patch updates our build files to use these new URLs.

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

No.

### How was this patch tested?

Existing build + tests.

Closes #27688 from JoshRosen/update-gcs-mirror-url.

Authored-by: Josh Rosen <joshrosen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-25 17:04:13 +09:00
Terry Kim 0fd4fa70c8 [SPARK-30885][SQL] V1 table name should be fully qualified if catalog name is provided
### What changes were proposed in this pull request?

For the following:
```
CREATE TABLE t USING json AS SELECT 1 AS i
SELECT * FROM spark_catalog.t
```
`spark_catalog.t` is resolved to `spark_catalog.default.t` assuming the current namespace is `default`. However, this is not consistent with V2 behavior where the namespace must be specified if the catalog name is provided. This PR proposes to fix this inconsistency.

### Why are the changes needed?

To be consistent with V2 table naming scheme in SQL commands.

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

Yes, now the user has to specify the namespace if the catalog name is provided. For example,
```
SELECT * FROM spark_catalog.t # Will throw AnalysisException with 'Session catalog cannot have an empty namespace: spark_catalog.t'
SELECT * FROM spark_catalog.default.t # OK
```

### How was this patch tested?

Added new tests

Closes #27642 from imback82/disallow_spark_catalog_wihtout_db.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 13:04:28 +08:00
Shixiong Zhu 3126557b07 [SPARK-30936][CORE] Set FAIL_ON_UNKNOWN_PROPERTIES to false by default to parse Spark events
### What changes were proposed in this pull request?

Set `FAIL_ON_UNKNOWN_PROPERTIES` to `false` in `JsonProtocol` to allow ignore unknown fields in a Spark event. After this change, if we add new fields to a Spark event parsed by `ObjectMapper`, the event json string generated by a new Spark version can still be read by an old Spark History Server.

Since Spark History Server is an extra service, it usually takes time to upgrade, and it's possible that a Spark application is upgraded before SHS. Forwards-compatibility will allow an old SHS to support new Spark applications (may lose some new features but most of functions should still work).

### Why are the changes needed?

`JsonProtocol` is supposed to provide strong backwards-compatibility and forwards-compatibility guarantees: any version of Spark should be able to read JSON output written by any other version, including newer versions.

However, the forwards-compatibility guarantee is broken for events parsed by `ObjectMapper`. If a new field is added to an event parsed by `ObjectMapper` (e.g., 6dc5921e66 (diff-dc5c7a41fbb7479cef48b67eb41ad254R33)), the event json string generated by a new Spark version cannot be parsed by an old version of SHS right now.

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

No.

### How was this patch tested?

The new added tests.

Closes #27680 from zsxwing/SPARK-30936.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 12:28:31 +08:00
Peter Toth 1a4e2423b2 [SPARK-30870][SQL] Column pruning shouldn't alias a nested column if it means the whole structure
### What changes were proposed in this pull request?
This PR fixes a bug in nested column aliasing by taking the data type of the referenced nested fields into account when calculating the number of extracted columns. After this PR this query runs without issues:
```
SELECT explodedvalue.*
FROM VALUES array(named_struct('nested', named_struct('a', 1, 'b', 2))) AS (value)
LATERAL VIEW explode(value) AS explodedvalue
```
This is a regression from Spark 2.4.

### Why are the changes needed?
To fix a bug.

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

### How was this patch tested?
Added new UT.

Closes #27675 from peter-toth/SPARK-30870.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-24 13:46:21 -08:00
Shixiong Zhu 293e5364e5 [SPARK-30927][SS] StreamingQueryManager should avoid keeping reference to terminated StreamingQuery
### What changes were proposed in this pull request?

Right now `StreamingQueryManager` will keep the last terminated query until `resetTerminated` is called. When the last terminated query has lots of states (a large sql plan, cached RDDs, etc.), it will keep a lot of memory unnecessarily. Actually, what `StreamingQueryManager` really needs is just the exception of the last failed query.

This PR changes the internal field `lastTerminatedQuery` in `StreamingQueryManager` to remember the last exception rather than the query to save the memory.

### Why are the changes needed?

Avoid keeping memory unnecessarily.

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

No

### How was this patch tested?

This PR doesn't change any public behaviors. The existing tests have covered the touched codes.

Closes #27678 from zsxwing/SPARK-30927.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 18:48:19 +09:00
beliefer 621e37e2ab [SPARK-28880][SQL] Support ANSI nested bracketed comments
### What changes were proposed in this pull request?
Spark SQL support single comments and bracketed comments now. This PR will support nested bracketed comments.

There are some mainstream database support the syntax.
**PostgreSQL:**
https://www.postgresql.org/docs/11/sql-syntax-lexical.html#SQL-SYNTAX-COMMENTS

**Vertica:**
https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Expressions/Comments.htm?zoom_highlight=comments

Note: Because Spark SQL not exists UT for single comments and bracketed comments, so I add some UT for them.

### Why are the changes needed?
nested bracketed comments is ANSI standard.

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

### How was this patch tested?
New UT

Closes #27495 from beliefer/nested-brancket-comments.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-24 00:28:46 -08:00
Peter Toth a372f76cbd [SPARK-30898][SQL] The behavior of MakeDecimal should not depend on SQLConf.get
### What changes were proposed in this pull request?
This PR adds a new `nullOnOverflow` parameter to `MakeDecimal` so as to avoid its value depending on `SQLConf.get` and change during planning.

### Why are the changes needed?
This allows to avoid the issue when the configuration change between different phases of planning, and this can silently break a query plan which can lead to crashes or data corruption.

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

### How was this patch tested?
Existing UTs.

Closes #27656 from peter-toth/SPARK-30898.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 16:48:48 +09:00
Peter Toth 612f63f39e [SPARK-30897][SQL] The behavior of ArrayExists should not depend on SQLConf.get
### What changes were proposed in this pull request?
This PR adds a new `followThreeValuedLogic` parameter to `ArrayExists` so as to avoid its value depending on `SQLConf.get` and change during planning.

### Why are the changes needed?
This allows to avoid the issue when the configuration change between different phases of planning, and this can silently break a query plan which can lead to crashes or data corruption.

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

### How was this patch tested?
Existing UTs.

Closes #27655 from peter-toth/SPARK-30897.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 16:47:08 +09:00
lijunqing 31bfbaeb90 [SPARK-30868][SQL] Throw Exception if runHive(sql) failed
### Why are the changes needed?
At present, HiveClientImpl.runHive will not throw an exception when it runs incorrectly, which will cause it to fail to feedback error information normally.
Example
```scala
spark.sql("add jar file:///tmp/not_exists.jar")
spark.sql("show databases").show()
```
/tmp/not_exists.jar doesn't exist, thus add jar is failed. However this code will run completely without causing application failure.

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

### How was this patch tested?
add new suite tests

Closes #27644 from stczwd/SPARK-30868.

Authored-by: lijunqing <lijunqing@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-24 15:42:45 +08:00
Burak Yavuz 4ff2718d54 [SPARK-30924][SQL][3.0] Add additional checks to Merge Into
### What changes were proposed in this pull request?

Merge Into is currently missing additional validation around:

 1. The lack of any WHEN statements
 2. The first WHEN MATCHED statement needs to have a condition if there are two WHEN MATCHED statements.
 3. Single use of UPDATE/DELETE

This PR introduces these validations.
(1) is required, because otherwise the MERGE statement is useless.
(2) is required, because otherwise the second WHEN MATCHED condition becomes dead code
(3) is up for debate, but the idea there is that a single expression should be sufficient to specify when you would like to update or delete your records. We restrict it for now to reduce surface area and ambiguity.

### Why are the changes needed?

To ease DataSource developers when building implementations for MERGE

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

Adds additional validation checks

### How was this patch tested?

Unit tests

Closes #27677 from brkyvz/mergeChecks.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-24 15:16:37 +08:00
jiake f4696ba252 [SPARK-30922][SQL] remove the max splits config in skewed join
### What changes were proposed in this pull request?
When skewed join optimization split more skewed readers, the plan may be very large and can not be shown in ui quickly. The config `spark.sql.adaptive.skewedJoinOptimization.skewedPartitionMaxSplits`  is to resolve the above ui shown issue. And after [PR#27493](https://github.com/apache/spark/pull/27493) combined the skewed readers into one, we not need this config.

### Why are the changes needed?
remove the unnecessary config

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

### How was this patch tested?
existing test

Closes #27673 from JkSelf/removeMaxSplitNum.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-24 14:29:25 +08:00
Maxim Gekk c41ef39819 [SPARK-30925][SQL] Prevent overflow/round errors in conversions of milliseconds to/from microseconds
### What changes were proposed in this pull request?
- Use `Math.multiplyExact()` in `DateTimeUtils.fromMillis()` to prevent silent overflow in conversion milliseconds to microseconds.
- Use `DateTimeUtils.fromMillis()` in all places where milliseconds are converted to microseconds
- Use `DateTimeUtils.toMillis()` in all places where microseconds are converted to milliseconds

### Why are the changes needed?

1. To prevent silent arithmetic overflow while multiplying by 1000 in `fromMillis()`. Instead of it, `new ArithmeticException("long overflow")` will be thrown, and handled accordantly.
2. To correctly round microseconds in conversion to milliseconds. For example, `1965-01-01 10:11:12.123456` is represented as `-157700927876544` in micro precision. In milliseconds precision the above needs to be represented as `-157700927877` or `1965-01-01 10:11:12.123`.

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

### How was this patch tested?
By `TimestampFormatterSuite`, `CastSuite`, `DateExpressionsSuite`, `IntervalExpressionsSuite`, `ExpressionParserSuite`, `ExpressionParserSuite`, `DateTimeUtilsSuite`, `IntervalUtilsSuite`

Closes #27676 from MaxGekk/millis-2-micros-overflow.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-24 14:06:25 +08:00
yi.wu 9c2eadc726 [SPARK-30844][SQL] Static partition should also follow StoreAssignmentPolicy when insert into table
### What changes were proposed in this pull request?

Make static partition also follows `StoreAssignmentPolicy` when insert into table:

if `StoreAssignmentPolicy=LEGACY`, using `Cast`;
if `StoreAssignmentPolicy=ANSI | STRIC`, using `AnsiCast`;

E.g., for the table `t` created by:

```
create table t(a int, b string) using parquet partitioned by (a)
```
and insert values with `StoreAssignmentPolicy=ANSI` using:
```
insert into t partition(a='ansi') values('ansi')
```

Before this PR:

```
+----+----+
|   b|   a|
+----+----+
|ansi|null|
+----+----+
```

After this PR, insert will fail by:
```
java.lang.NumberFormatException: invalid input syntax for type numeric: ansi
```

(It should be better if we could use `TableOutputResolver.checkField` to fully follow `StoreAssignmentPolicy`. But since we lost the data type of static partition's value at first place, it's hard to use `TableOutputResolver.checkField`.)

### Why are the changes needed?

I think we should follow `StoreAssignmentPolicy` when insert into table for any columns, including static partition.

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

No.

### How was this patch tested?

Added new test.

Closes #27597 from Ngone51/fix-static-partition.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-02-23 17:46:19 +09:00
yi.wu 25f5bfaa6e [SPARK-30903][SQL] Fail fast on duplicate columns when analyze columns
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->

Add new `CommandCheck` rule and fail fast when detects duplicate columns in `AnalyzeColumnCommand`.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->

To avoid duplicate statistics computation for the same column in `AnalyzeColumnCommand`.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->

Yes. User now get exception when input duplicate columns.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->

Added new test.

Closes #27651 from Ngone51/fail_on_dup_cols.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-02-23 09:52:54 +09:00
Christian Stuart bcce1b1040 [SPARK-30904][SQL] Thrift RowBasedSet serialization throws NullPointerException on NULL BigDecimal
### What changes were proposed in this pull request?

This PR fixes SPARK-30904 by adding a null check.

### Why are the changes needed?

For HIVE_CLI_SERVICE_PROTOCOL_V5 and below, serialization fails on NULL-containing decimal columns, caused by a call to  `value.toPlainString()`, where `value` might be null. This null check fixes it.

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

No

### How was this patch tested?

A test was added for serialization of NULL decimals for all HIVE_CLI_SERVICE_PROTOCOL versions.

Closes #27654 from CJStuart/SPARK-30904.

Authored-by: Christian Stuart <christian.stuart@databricks.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-02-21 21:39:35 -07:00
Maxim Gekk 310c14ac8d [MINOR][SQL] Add a comment for removedSQLConfigs
### What changes were proposed in this pull request?
In the PR, I propose to explain in the description of `removedSQLConfigs` when removed SQL configs should NOT be placed to the map.

### Why are the changes needed?
To make the cases when SQL configs should be added to `removedSQLConfigs` more clear. Recently, `spark.sql.variable.substitute.depth` was removed from the map by #27646 because it contradicts to the condition described by the PR.

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

### How was this patch tested?
By `./dev/scalastyle`

Closes #27653 from MaxGekk/removedSQLConfigs-comment.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-22 09:48:10 +09:00
beliefer 59d6d5cbb0 [SPARK-30840][CORE][SQL] Add version property for ConfigEntry and ConfigBuilder
### What changes were proposed in this pull request?
Spark `ConfigEntry` and `ConfigBuilder` missing Spark version information of each configuration at release. This is not good for Spark user when they visiting the page of spark configuration.
http://spark.apache.org/docs/latest/configuration.html
The new Spark SQL config docs looks like:
![sql配置截屏](https://user-images.githubusercontent.com/8486025/74604522-cb882f00-50f9-11ea-8683-57a90f9e3347.png)

```
> SET -v
spark.sql.adaptive.enabled      false   When true, enable adaptive query execution.
spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin       0.2     The relation with a non-empty partition ratio lower than this config will not be considered as the build side of a broadcast-hash join in adaptive execution regardless of its size.This configuration only has an effect when 'spark.sql.adaptive.enabled' is enabled.
spark.sql.adaptive.optimizeSkewedJoin.enabled   true    When true and adaptive execution is enabled, a skewed join is automatically handled at runtime.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor     10      A partition is considered as a skewed partition if its size is larger than this factor multiple the median partition size and also larger than  spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionMaxSplits  5       Configures the maximum number of task to handle a skewed partition in adaptive skewedjoin.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold      64MB    Configures the minimum size in bytes for a partition that is considered as a skewed partition in adaptive skewed join.
spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled    true    Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks one by one, fetching continuous shuffle blocks for the same map task in batch can reduce IO and improve performance. Note, multiple continuous blocks exist in single fetch request only happen when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled, this feature also depends on a relocatable serializer, the concatenation support codec in use and the new version shuffle fetch protocol.
spark.sql.adaptive.shuffle.localShuffleReader.enabled   true    When true and 'spark.sql.adaptive.enabled' is enabled, this enables the optimization of converting the shuffle reader to local shuffle reader for the shuffle exchange of the broadcast hash join in probe side.
spark.sql.adaptive.shuffle.maxNumPostShufflePartitions  <undefined>     The advisory maximum number of post-shuffle partitions used in adaptive execution. This is used as the initial number of pre-shuffle partitions. By default it equals to spark.sql.shuffle.partitions. This configuration only has an effect when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled.
```

**Note**: Because there are so many configuration items that are exposed and require a lot of finishing, I will add the version numbers of these configuration items in another PR.

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27592 from beliefer/add-version-to-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-22 09:46:42 +09:00
Eric Wu 1f0300fb16 [SPARK-30764][SQL] Improve the readability of EXPLAIN FORMATTED style
### What changes were proposed in this pull request?
The style of `EXPLAIN FORMATTED` output needs to be improved. We’ve already got some observations/ideas in
https://github.com/apache/spark/pull/27368#discussion_r376694496
https://github.com/apache/spark/pull/27368#discussion_r376927143

Observations/Ideas:
1. Using comma as the separator is not clear, especially commas are used inside the expressions too.
2. Show the column counts first? For example, `Results [4]: …`
3. Currently the attribute names are automatically generated, this need to refined.
4. Add arguments field in common implementations as `EXPLAIN EXTENDED` did by calling `argString` in `TreeNode.simpleString`. This will eliminate most existing minor differences between
`EXPLAIN EXTENDED` and `EXPLAIN FORMATTED`.
5. Another improvement we can do is: the generated alias shouldn't include attribute id. collect_set(val, 0, 0)#123 looks clearer than collect_set(val#456, 0, 0)#123

This PR is currently addressing comments 2 & 4, and open for more discussions on improving readability.

### Why are the changes needed?
The readability of `EXPLAIN FORMATTED` need to be improved, which will help user better understand the query plan.

### Does this PR introduce any user-facing change?
Yes, `EXPLAIN FORMATTED` output style changed.

### How was this patch tested?
Update expect results of test cases in explain.sql

Closes #27509 from Eric5553/ExplainFormattedRefine.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 23:36:14 +08:00
maryannxue 6058ce97b9 [SPARK-30906][SQL] Turning off AQE in CacheManager is not thread-safe
### What changes were proposed in this pull request?
This PR aims to fix the thread-safety issue in turning off AQE for CacheManager by cloning the current session and changing the AQE conf on the cloned session.
This PR also adds a utility function for cloning the session with AQE disabled conf value, which can be shared by another caller.

### Why are the changes needed?
To fix the potential thread-unsafe problem.

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

### How was this patch tested?
Manually tested CachedTableSuite with AQE settings enabled.

Closes #27659 from maryannxue/spark-30906.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 22:49:20 +08:00
Yuanjian Li a5efbb284e [SPARK-30809][SQL] Review and fix issues in SQL API docs
### What changes were proposed in this pull request?
- Add missing `since` annotation.
- Don't show classes under `org.apache.spark.sql.dynamicpruning` package in API docs.
- Fix the scope of `xxxExactNumeric` to remove it from the API docs.

### Why are the changes needed?
Avoid leaking APIs unintentionally in Spark 3.0.0.

### Does this PR introduce any user-facing change?
No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?
Manually generated the API docs and verified the above issues have been fixed.

Closes #27560 from xuanyuanking/SPARK-30809.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 17:03:22 +08:00
Maxim Gekk abe0821ee9 [SPARK-30894][SQL] Make Size's nullable independent from SQL config changes
### What changes were proposed in this pull request?
In the PR, I propose to add the `legacySizeOfNull ` parameter to the `Size` expression, and pass the value of `spark.sql.legacy.sizeOfNull` if `legacySizeOfNull` is not provided on creation of `Size`.

### Why are the changes needed?
This allows to avoid the issue when the configuration change between different phases of planning, and this can silently break a query plan which can lead to crashes or data corruption.

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

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

Closes #27658 from MaxGekk/Size-SQLConf-get-deps.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 15:32:11 +08:00
yi.wu 82ce4753aa [SPARK-26580][SQL][ML][FOLLOW-UP] Throw exception when use untyped UDF by default
### What changes were proposed in this pull request?

This PR proposes to throw exception by default when user use untyped UDF(a.k.a `org.apache.spark.sql.functions.udf(AnyRef, DataType)`).

And user could still use it by setting `spark.sql.legacy.useUnTypedUdf.enabled` to `true`.

### Why are the changes needed?

According to #23498, since Spark 3.0, the untyped UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will  return 0 in Spark 3.0 but null in Spark 2.4. And the behavior change is introduced due to Spark3.0 is built with Scala 2.12 by default.

As a result, this might change data silently and may cause correctness issue if user still expect `null` in some cases. Thus, we'd better to encourage user to use typed UDF to avoid this problem.

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

Yeah. User will hit exception now when use untyped UDF.

### How was this patch tested?

Added test and updated some tests.

Closes #27488 from Ngone51/spark_26580_followup.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 14:46:54 +08:00
yi.wu 4d356554a6 [MINOR][SQL] Fix error position of NOSCAN
### What changes were proposed in this pull request?

Point to correct position when miswrite `NOSCAN` detects.

### Why are the changes needed?

Before:

```
[info]   org.apache.spark.sql.catalyst.parser.ParseException: Expected `NOSCAN` instead of `SCAN`(line 1, pos 0)
[info]
[info] == SQL ==
[info] ANALYZE TABLE analyze_partition_with_null PARTITION (name) COMPUTE STATISTICS SCAN
[info] ^^^
```

After:

```
[info]   org.apache.spark.sql.catalyst.parser.ParseException: Expected `NOSCAN` instead of `SCAN`(line 1, pos 78)
[info]
[info] == SQL ==
[info] ANALYZE TABLE analyze_partition_with_null PARTITION (name) COMPUTE STATISTICS SCAN
[info] ------------------------------------------------------------------------------^^^
```

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

Yes, user will see better error message.

### How was this patch tested?

Manually test.

Closes #27662 from Ngone51/fix_noscan_reference.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-02-21 15:21:53 +09:00
wuyi 5eb004f4bb Revert "[SPARK-28093][SQL] Fix TRIM/LTRIM/RTRIM function parameter order issue"
### What changes were proposed in this pull request?

This reverts commit bef5d9d6c3.

### Why are the changes needed?

Revert it according to https://github.com/apache/spark/pull/24902#issuecomment-584511167.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27540 from Ngone51/revert_spark_28093.

Lead-authored-by: wuyi <yi.wu@databricks.com>
Co-authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 12:55:32 +08:00
Maxim Gekk bb40ab09f4 [SPARK-30892][SQL] Exclude spark.sql.variable.substitute.depth from removedSQLConfigs
### What changes were proposed in this pull request?

Exclude the SQL config `spark.sql.variable.substitute.depth` from `SQLConf.removedSQLConfigs`

### Why are the changes needed?
By the #27169, the config was placed to `SQLConf.removedSQLConfigs`. And as a consequence of that when an user set it non-default value (1 for example),  he/she will get an exception. It is acceptable for SQL configs that could impact on the behavior but not for this particular config. Raising of such exception will just make migration to Spark 3.0 more difficult.

### Does this PR introduce any user-facing change?
Yes, before the changes users get an exception when he/she set `spark.sql.variable.substitute.depth` to a value different from `40`.

### How was this patch tested?
Run `spark.conf.set("spark.sql.variable.substitute.depth", 1)` in `spark-shell`.

Closes #27646 from MaxGekk/remove-substitute-depth-conf.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 00:44:09 +08:00
Maxim Gekk a551715fd2 [SPARK-29930][SPARK-30416][SQL][FOLLOWUP] Move deprecated/removed config checks from RuntimeConfig to SQLConf
### What changes were proposed in this pull request?
- Output warnings for deprecated SQL configs in `SQLConf. setConfWithCheck()` and in `SQLConf. unsetConf()`
- Throw an exception for removed SQL configs in `SQLConf. setConfWithCheck()` when they set to non-default values
- Remove checking of deprecated and removed SQL configs from RuntimeConfig

### Why are the changes needed?
Currently, warnings/exceptions are printed only when a SQL config is set dynamically, for instance via `spark.conf.set()`. After the changes, removed/deprecated SQL configs will be checked when they set statically. For example:
```
$ bin/spark-shell --conf spark.sql.fromJsonForceNullableSchema=false
scala> spark.emptyDataFrame
java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':
...
Caused by: org.apache.spark.sql.AnalysisException: The SQL config 'spark.sql.fromJsonForceNullableSchema' was removed in the version 3.0.0. It was removed to prevent errors like SPARK-23173 for non-default value.
```
```
$ bin/spark-shell --conf spark.sql.hive.verifyPartitionPath=false
scala> spark.emptyDataFrame
20/02/20 02:10:26 WARN SQLConf: The SQL config 'spark.sql.hive.verifyPartitionPath' has been deprecated in Spark v3.0 and may be removed in the future. This config is replaced by 'spark.files.ignoreMissingFiles'.
```

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

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

Closes #27645 from MaxGekk/remove-sql-configs-followup-2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 00:00:48 +08:00
Wenchen Fan 704d249a56 [SPARK-26071][FOLLOWUP] Improve migration guide of disallowing map type map key
### What changes were proposed in this pull request?

mention the workaround if users do want to use map type as key, and add a test to demonstrate it.

### Why are the changes needed?

it's better to provide an alternative when we ban something.

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

no

### How was this patch tested?

N/A

Closes #27621 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-20 22:10:04 +08:00
Maxim Gekk 4248b7fbb9 [SPARK-30858][SQL] Make IntegralDivide's dataType independent from SQL config changes
### What changes were proposed in this pull request?
In the PR, I propose to add the `returnLong` parameter to `IntegralDivide`, and pass the value of `spark.sql.legacy.integralDivide.returnBigint` if `returnLong` is not provided on creation of `IntegralDivide`.

### Why are the changes needed?
This allows to avoid the issue when the configuration change between different phases of planning, and this can silently break a query plan which can lead to crashes or data corruption.

OptionsAttachments

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

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

Closes #27628 from MaxGekk/integral-divide-conf.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-20 21:26:20 +08:00
Gengliang Wang 92d5d40c8e [SPARK-30881][SQL][DOCS] Revise the doc of spark.sql.sources.parallelPartitionDiscovery.threshold
### What changes were proposed in this pull request?

Revise the doc of SQL configuration `spark.sql.sources.parallelPartitionDiscovery.threshold`.
### Why are the changes needed?

The doc of configuration "spark.sql.sources.parallelPartitionDiscovery.threshold" is not accurate on the part "This applies to Parquet, ORC, CSV, JSON and LibSVM data sources".

We should revise it as effective on all the file-based data sources.

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

No

### How was this patch tested?

None. It's just doc.

Closes #27639 from gengliangwang/reviseParallelPartitionDiscovery.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-20 00:59:22 -08:00
herman c92d437c46 [SPARK-30811][SQL] CTE should not cause stack overflow when it refers to non-existent table with same name
### Why are the changes needed?
This ports the tests introduced in 7285eea683 to master to avoid future regressions.

### Background
A query with Common Table Expressions can cause a stack overflow when it contains a CTE that refers a non-existing table with the same name. The name of the table need to have a database qualifier. This is caused by a couple of things:

- CTESubstitution runs analysis on the CTE, but this does not throw an exception because the table has a database qualifier. The reason is that we don't fail is because we re-attempt to resolve the relation in a later rule;
- CTESubstitution replace logic does not check if the table it is replacing has a database, it shouldn't replace the relation if it does. So now we will happily replace nonexist.t with t;

Note that this not an issue for master or the spark-3.0 branch.

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

### How was this patch tested?
Added regression test to `AnalysisErrorSuite` and `DataFrameSuite`.

Closes #27635 from hvanhovell/SPARK-30811-master.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-19 10:17:46 -08:00
Nicholas Chammas 2ab8d674ba [SPARK-30731] Update deprecated Mkdocs option
Split from #27534.

### What changes were proposed in this pull request?

This PR updates a deprecated Mkdocs option to use the new name.

### Why are the changes needed?

This change will prevent the docs from failing to build when we update to a version of Mkdocs that no longer supports the deprecated option.

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

No.

### How was this patch tested?

I built the docs locally and reviewed them in my browser.

Closes #27626 from nchammas/SPARK-30731-mkdocs-dep-opt.

Authored-by: Nicholas Chammas <nicholas.chammas@liveramp.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-19 17:28:58 +09:00
jiake 10a4eafcfe [SPARK-30812][SQL] update the skew join configs by adding the prefix "skewedJoinOptimization"
### What changes were proposed in this pull request?
This is a follow up in [PR#27563](https://github.com/apache/spark/pull/27563).
This PR adds the prefix of "skewedJoinOptimization" in the skew join related configs.

### Why are the changes needed?
address remaining address

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

### How was this patch tested?
only update config and no need new ut.

Closes #27630 from JkSelf/renameskewjoinconfig.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-19 15:55:29 +08:00
LantaoJin c0715221b2 [SPARK-30785][SQL] Create table like should keep tracksPartitionsInCatalog same with source table
### What changes were proposed in this pull request?
Table generated by `CREATE TABLE LIKE` a partitioned table is a partitioned table. But when run `ALTER TABLE ADD PARTITION`, it will throw `AnalysisException: ALTER TABLE ADD PARTITION is not allowed`. That's because the default value of `tracksPartitionsInCatalog` from `CREATE TABLE LIKE` always is false.

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

### How was this patch tested?
Add a unit test.

Closes #27538 from LantaoJin/SPARK-30785.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-19 15:05:34 +08:00
beliefer 0894dbab2c [MINOR][SQL] Improve readability for window execution
### What changes were proposed in this pull request?
I read the comments of `WindowExec` and found some comment will cause confusion and another need to improve.

### Why are the changes needed?
This PR will enhance the readability and let developer works more easy

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

### How was this patch tested?
No need

Closes #27431 from beliefer/improve-window-readability.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-19 14:26:27 +08:00
Wenchen Fan 1b67d546bd revert SPARK-29663 and SPARK-29688
### What changes were proposed in this pull request?

This PR reverts https://github.com/apache/spark/pull/26325 and https://github.com/apache/spark/pull/26347

### Why are the changes needed?

When we do sum/avg, we need a wider type of input to hold the sum value, to reduce the possibility of overflow. For example, we use long to hold the sum of integral inputs, use double to hold the sum of float/double.

However, we don't have a wider type of interval. Also the semantic is unclear: what if the days field overflows but the months field doesn't? Currently the avg of `1 month` and `2 month` is `1 month 15 days`, which assumes 1 month has 30 days and we should avoid this assumption.

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

yes, remove 2 features added in 3.0

### How was this patch tested?

N/A

Closes #27619 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-18 21:19:57 +01:00
yi.wu 68d7edf949 [SPARK-30812][SQL][CORE] Revise boolean config name to comply with new config naming policy
### What changes were proposed in this pull request?

Revise below config names to comply with [new config naming policy](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html):

SQL:
* spark.sql.execution.subquery.reuse.enabled / [SPARK-27083](https://issues.apache.org/jira/browse/SPARK-27083)
* spark.sql.legacy.allowNegativeScaleOfDecimal.enabled / [SPARK-30252](https://issues.apache.org/jira/browse/SPARK-30252)
* spark.sql.adaptive.optimizeSkewedJoin.enabled / [SPARK-29544](https://issues.apache.org/jira/browse/SPARK-29544)
* spark.sql.legacy.property.nonReserved / [SPARK-30183](https://issues.apache.org/jira/browse/SPARK-30183)
* spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled / [SPARK-26389](https://issues.apache.org/jira/browse/SPARK-26389)
* spark.sql.analyzer.failAmbiguousSelfJoin.enabled / [SPARK-28344](https://issues.apache.org/jira/browse/SPARK-28344)
* spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled / [SPARK-30074](https://issues.apache.org/jira/browse/SPARK-30074)
* spark.sql.execution.pandas.arrowSafeTypeConversion / [SPARK-25811](https://issues.apache.org/jira/browse/SPARK-25811)
* spark.sql.legacy.looseUpcast / [SPARK-24586](https://issues.apache.org/jira/browse/SPARK-24586)
* spark.sql.legacy.arrayExistsFollowsThreeValuedLogic / [SPARK-28052](https://issues.apache.org/jira/browse/SPARK-28052)
* spark.sql.sources.ignoreDataLocality.enabled / [SPARK-29189](https://issues.apache.org/jira/browse/SPARK-29189)
* spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled / [SPARK-9853](https://issues.apache.org/jira/browse/SPARK-9853)

CORE:
* spark.eventLog.erasureCoding.enabled / [SPARK-25855](https://issues.apache.org/jira/browse/SPARK-25855)
* spark.shuffle.readHostLocalDisk.enabled / [SPARK-30235](https://issues.apache.org/jira/browse/SPARK-30235)
* spark.scheduler.listenerbus.logSlowEvent.enabled / [SPARK-29001](https://issues.apache.org/jira/browse/SPARK-29001)
* spark.resources.coordinate.enable / [SPARK-27371](https://issues.apache.org/jira/browse/SPARK-27371)
* spark.eventLog.logStageExecutorMetrics.enabled / [SPARK-23429](https://issues.apache.org/jira/browse/SPARK-23429)

### Why are the changes needed?

To comply with the config naming policy.

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

No. Configurations listed above are all newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27563 from Ngone51/revise_boolean_conf_name.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 20:39:50 +08:00
yi.wu 643a480b11 [SPARK-30863][SQL] Distinguish Cast and AnsiCast in toString
### What changes were proposed in this pull request?

Prefix by `ansi_`  in `toString` if it's a `AnsiCast` or ansi enabled `Cast`.

E.g. run `spark.sql("select cast('51' as int)").queryExecution.analyzed` under ansi mode.

Before this PR:
```
Project [cast(51 as int) AS CAST(51 AS INT)#0]
+- OneRowRelation
```

After this PR:
```
Project [ansi_cast(51 as int) AS CAST(51 AS INT)#0]
+- OneRowRelation
```

### Why are the changes needed?

This is useful while comparing `LogicalPlan`s literally.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27608 from Ngone51/ansi_cast_tostring.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 16:10:43 +08:00
HyukjinKwon 9618806f44 [SPARK-30847][SQL] Take productPrefix into account in MurmurHash3.productHash
### What changes were proposed in this pull request?

This PR proposes to port Scala's bugfix https://github.com/scala/scala/pull/7693 (Scala 2.13) to address https://github.com/scala/bug/issues/10495 issue.

In short, it is possible for different product instances having the same children to have the same hash. See:

```scala
scala> spark.range(1).selectExpr("id - 1").queryExecution.analyzed.semanticHash()
res0: Int = -565572825

scala> spark.range(1).selectExpr("id + 1").queryExecution.analyzed.semanticHash()
res1: Int = -565572825
```

### Why are the changes needed?

It was found during the review of https://github.com/apache/spark/pull/27565. We should better produce different hash for different objects.

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

No, it's not identified. Possibly performance related issue.

### How was this patch tested?

Manually tested, and unittest was added.

Closes #27601 from HyukjinKwon/SPARK-30847.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 14:28:44 +08:00
Terry Kim 5866bc77d7 [SPARK-30814][SQL] ALTER TABLE ... ADD COLUMN position should be able to reference columns being added
### What changes were proposed in this pull request?

In ALTER TABLE, a column in ADD COLUMNS can depend on the position of a column that is just being added. For example, for a table with the following schema:
```
root:
  - a: string
  - b: long
```
, the following should work:
```
ALTER TABLE t ADD COLUMNS (x int AFTER a, y int AFTER x)
```
Currently, the above statement will throw an exception saying that AFTER x cannot be resolved, because x doesn't exist yet. This PR proposes to fix this issue.

### Why are the changes needed?

To fix a bug described above.

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

Yes, now
```
ALTER TABLE t ADD COLUMNS (x int AFTER a, y int AFTER x)
```
works as expected.

### How was this patch tested?

Added new tests

Closes #27584 from imback82/alter_table_pos_fix.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 13:01:45 +08:00
Liang Zhang d8c0599e54 [SPARK-30791][SQL][PYTHON] Add 'sameSemantics' and 'sementicHash' methods in Dataset
### What changes were proposed in this pull request?
This PR added two DeveloperApis to the Dataset[T] class. Both methods are just exposing lower-level methods to the Dataset[T] class.

### Why are the changes needed?
They are useful for checking whether two dataframes are the same when implementing dataframe caching in python, and also get a unique ID. It's easier to use if we wrap the lower-level APIs.

### Does this PR introduce any user-facing change?
```
scala> val df1 = Seq((1,2),(4,5)).toDF("col1", "col2")
df1: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df2 = Seq((1,2),(4,5)).toDF("col1", "col2")
df2: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df3 = Seq((0,2),(4,5)).toDF("col1", "col2")
df3: org.apache.spark.sql.DataFrame = [col1: int, col2: int]

scala> val df4 = Seq((0,2),(4,5)).toDF("col0", "col2")
df4: org.apache.spark.sql.DataFrame = [col0: int, col2: int]

scala> df1.semanticHash
res0: Int = 594427822

scala> df2.semanticHash
res1: Int = 594427822

scala> df1.sameSemantics(df2)
res2: Boolean = true

scala> df1.sameSemantics(df3)
res3: Boolean = false

scala> df3.semanticHash
res4: Int = -1592702048

scala> df4.semanticHash
res5: Int = -1592702048

scala> df4.sameSemantics(df3)
res6: Boolean = true
```

### How was this patch tested?
Unit test in scala and doctest in python.

Note: comments are copied from the corresponding lower-level APIs.
Note: There are some issues to be fixed that would improve the hash collision rate: https://github.com/apache/spark/pull/27565#discussion_r379881028

Closes #27565 from liangz1/df-same-result.

Authored-by: Liang Zhang <liang.zhang@databricks.com>
Signed-off-by: WeichenXu <weichen.xu@databricks.com>
2020-02-18 09:22:26 +08:00
Nicholas Chammas 4ed9b88996 [SPARK-30832][DOCS] SQL function doc headers should link to anchors
### Why are the changes needed?

In most of our docs, you can click on a heading to immediately get an anchor link to that specific section of the docs. This is very handy when you are reading the docs and want to share a link to a specific part.

The SQL function docs are lacking this. This PR adds this convenience to the SQL function docs.

Here's the impact on the generated HTML.

Before this PR:

```html
<h3 id="array_join">array_join</h3>
```

After this PR:

```html
<h3 id="array_join"><a class="toclink" href="#array_join">array_join</a></h3>
```

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

No.

### How was this patch tested?

I built the docs manually and reviewed the results in my browser.

Closes #27585 from nchammas/SPARK-30832-sql-doc-headers.

Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-02-17 19:19:28 -06:00
beliefer d8d3ce5c76 [SPARK-30825][SQL][DOC] Update documents information for window function
### What changes were proposed in this pull request?
I checked the all the window function and found all of them not add parameter information and version information to the document.
This PR will make a supplement.

### Why are the changes needed?
Documentation is missing and does not meet new standards.

### Does this PR introduce any user-facing change?
Yes. User will face the information of parameters and version.

### How was this patch tested?
Exists UT

Closes #27572 from beliefer/add_since_for_window_function.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 09:44:34 +09:00
Ajith 657d151395 [SPARK-29174][SQL] Support LOCAL in INSERT OVERWRITE DIRECTORY to data source
### What changes were proposed in this pull request?
`INSERT OVERWRITE LOCAL DIRECTORY` is supported with ensuring the provided path is always using `file://` as scheme and removing the check which throws exception if we do insert overwrite by mentioning directory with `LOCAL` syntax

### Why are the changes needed?
without the modification in PR, ``` insert overwrite local directory <location> using ```

throws exception

```
Error: org.apache.spark.sql.catalyst.parser.ParseException:

LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source(line 1, pos 0)
```
which was introduced in https://github.com/apache/spark/pull/18975, but this restriction is not needed, hence dropping the same.
Keep behaviour consistent for local and remote file-system in  `INSERT OVERWRITE DIRECTORY`

### Does this PR introduce any user-facing change?
Yes, after this change `INSERT OVERWRITE LOCAL DIRECTORY` will not throw exception

### How was this patch tested?
Added UT

Closes #27039 from ajithme/insertoverwrite2.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-18 09:42:31 +09:00
Ajith 2854091d12 [SPARK-22590][SQL] Copy sparkContext.localproperties to child thread in BroadcastExchangeExec.executionContext
### What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the broadcast execution thread in `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#executionContext`

### Why are the changes needed?
When executing `BroadcastExchangeExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads.
These threads are created in the executionContext (thread pools). Each Thread pool has a default `keepAliveSeconds` of 60 seconds for idle threads.
Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob`

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

### How was this patch tested?
Added UT

Closes #27266 from ajithme/broadcastlocalprop.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 02:26:52 +08:00
Maxim Gekk afaeb29599 [SPARK-30808][SQL] Enable Java 8 time API in Thrift server
### What changes were proposed in this pull request?
- Set `spark.sql.datetime.java8API.enabled` to `true` in `hiveResultString()`, and restore it back at the end of the call.
- Convert collected `java.time.Instant` & `java.time.LocalDate` to `java.sql.Timestamp` and `java.sql.Date` for correct formatting.

### Why are the changes needed?
Because of textual representation of timestamps/dates before 1582 year is incorrect:
```shell
$ export TZ="America/Los_Angeles"
$ ./bin/spark-sql -S
```
```sql
spark-sql> set spark.sql.session.timeZone=America/Los_Angeles;
spark.sql.session.timeZone	America/Los_Angeles
spark-sql> SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20');
1001-01-01 00:07:02
```
It must be 1001-01-01 00:**00:00**.

### Does this PR introduce any user-facing change?
Yes. After the changes:
```shell
$ export TZ="America/Los_Angeles"
$ ./bin/spark-sql -S
```
```sql
spark-sql> set spark.sql.session.timeZone=America/Los_Angeles;
spark.sql.session.timeZone	America/Los_Angeles
spark-sql> SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20');
1001-01-01 00:00:00
```

### How was this patch tested?
By running hive-thiftserver tests. In particular:
```
./build/sbt -Phadoop-2.7 -Phive-2.3 -Phive-thriftserver "hive-thriftserver/test:testOnly *SparkThriftServerProtocolVersionsSuite"
```

Closes #27552 from MaxGekk/hive-thriftserver-java8-time-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 02:15:44 +08:00
yi.wu a1d536cb3e [SPARK-15616][FOLLOW-UP][SQL] Sub Optimizer should include super.postHocOptimizationBatches
### What changes were proposed in this pull request?

Let sub optimizer's  `postHocOptimizationBatches` also includes super's `postHocOptimizationBatches`.

### Why are the changes needed?

It's necessary according to the design of catalyst optimizer.

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

No.

### How was this patch tested?

Pass jenkins.

Closes #27607 from Ngone51/spark_15616_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 01:52:23 +08:00
Yuanjian Li 5ffc5ff55e [SPARK-11150][SQL][FOLLOWUP] Move sql/dynamicpruning to sql/execution/dynamicpruning
### What changes were proposed in this pull request?
Follow-up work for #25600. In this PR, we move `sql/dynamicpruning` to `sql/execution/dynamicpruning`.

### Why are the changes needed?
Fix the unexpected public APIs in 3.0.0 #27560.

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

### How was this patch tested?
Existing UT.

Closes #27581 from xuanyuanking/SPARK-11150-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 01:44:14 +08:00
Yuming Wang 76ddb6d835 [SPARK-30755][SQL] Update migration guide and add actionable exception for HIVE-15167
### What changes were proposed in this pull request?
[HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) removed the `SerDe` interface. This may break custom `SerDe` builds for Hive 1.2. This PR update the migration guide for this change.

### Why are the changes needed?

Otherwise:
```
2020-01-27 05:11:20.446 - stderr> 20/01/27 05:11:20 INFO DAGScheduler: ResultStage 2 (main at NativeMethodAccessorImpl.java:0) failed in 1.000 s due to Job aborted due to stage failure: Task 0 in stage 2.0 failed 4 times, most recent failure: Lost task 0.3 in stage 2.0 (TID 13, 10.110.21.210, executor 1): java.lang.NoClassDefFoundError: org/apache/hadoop/hive/serde2/SerDe
  2020-01-27 05:11:20.446 - stderr>  at java.lang.ClassLoader.defineClass1(Native Method)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.ClassLoader.defineClass(ClassLoader.java:756)
  2020-01-27 05:11:20.446 - stderr>  at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
  2020-01-27 05:11:20.446 - stderr>  at java.net.URLClassLoader.defineClass(URLClassLoader.java:468)
  2020-01-27 05:11:20.446 - stderr>  at java.net.URLClassLoader.access$100(URLClassLoader.java:74)
  2020-01-27 05:11:20.446 - stderr>  at java.net.URLClassLoader$1.run(URLClassLoader.java:369)
  2020-01-27 05:11:20.446 - stderr>  at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
  2020-01-27 05:11:20.446 - stderr>  at java.security.AccessController.doPrivileged(Native Method)
  2020-01-27 05:11:20.446 - stderr>  at java.net.URLClassLoader.findClass(URLClassLoader.java:362)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
  2020-01-27 05:11:20.446 - stderr>  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.ClassLoader.loadClass(ClassLoader.java:405)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.Class.forName0(Native Method)
  2020-01-27 05:11:20.446 - stderr>  at java.lang.Class.forName(Class.java:348)
  2020-01-27 05:11:20.446 - stderr>  at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializerClass(TableDesc.java:76)
.....
```

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

### How was this patch tested?
Manual test

Closes #27492 from wangyum/SPARK-30755.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-17 09:26:56 -08:00
wangguangxin.cn 0ae3ff60c4 [SPARK-30806][SQL] Evaluate once per group in UnboundedWindowFunctionFrame
### What changes were proposed in this pull request?
We only need to do aggregate evaluation once per group in `UnboundedWindowFunctionFrame`

### Why are the changes needed?
Currently, in `UnboundedWindowFunctionFrame.write`,it re-evaluate the processor for each row in a group, which is not necessary in fact which I'll address later. It hurts performance when the evaluation is time-consuming (for example, Percentile's eval need to sort its buffer and do some calculation). In our production, there is a percentile with window operation sql,  it costs more than 10 hours in SparkSQL while 10min in Hive.

In fact, `UnboundedWindowFunctionFrame` can be treated as `SlidingWindowFunctionFrame` with `lbound = UnboundedPreceding` and `ubound = UnboundedFollowing`, just as its comments. In that case, `SlidingWindowFunctionFrame` also only do evaluation once for each group.

The performance issue can be reproduced by running the follow scripts in local spark-shell
```
spark.range(100*100).map(i => (i, "India")).toDF("uv", "country").createOrReplaceTempView("test")
sql("select uv, country, percentile(uv, 0.95) over (partition by country) as ptc95 from test").collect.foreach(println)
```
Before this patch, the sql costs **128048 ms**.
With this patch,  the sql costs **3485 ms**.

If we increase the data size to 1000*1000 for example, then spark cannot even produce result without this patch(I'v waited for several hours).

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

### How was this patch tested?
Existing UT

Closes #27558 from WangGuangxin/windows.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-17 18:15:54 +01:00
Yuanjian Li e4a541b278 [SPARK-30829][SQL] Define LegacyBehaviorPolicy enumeration as the common value for result change configs
### What changes were proposed in this pull request?
Define a new enumeration `LegacyBehaviorPolicy` in SQLConf, it will be used as the common value for result change configs.

### Why are the changes needed?
During API auditing for the 3.0 release, we found several new approaches that will change the results silently. For these features, we need a common three-value config.

### Does this PR introduce any user-facing change?
Yes, original config `spark.sql.legacy.ctePrecedence.enabled` change to `spark.sql.legacy.ctePrecedencePolicy`.

### How was this patch tested?
Existing UT.

Closes #27579 from xuanyuanking/SPARK-30829.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 00:52:05 +08:00
Arwin Tio 25e9156bc0 [SPARK-29089][SQL] Parallelize blocking FileSystem calls in DataSource#checkAndGlobPathIfNecessary
### What changes were proposed in this pull request?
See JIRA: https://issues.apache.org/jira/browse/SPARK-29089
Mailing List: http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrameReader-bottleneck-in-DataSource-checkAndGlobPathIfNecessary-when-reading-S3-files-td27828.html

When using DataFrameReader#csv to read many files on S3, globbing and fs.exists on DataSource#checkAndGlobPathIfNecessary becomes a bottleneck.

From the mailing list discussions, an improvement that can be made is to parallelize the blocking FS calls:

> - have SparkHadoopUtils differentiate between files returned by globStatus(), and which therefore exist, and those which it didn't glob for -it will only need to check those.
> - add parallel execution to the glob and existence checks

### Why are the changes needed?

Verifying/globbing files happens on the driver, and if this operations take a long time (for example against S3), then the entire cluster has to wait, potentially sitting idle. This change hopes to make this process faster.

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

No

### How was this patch tested?

I added a test suite `DataSourceSuite` - open to suggestions for better naming.

See [here](https://github.com/apache/spark/pull/25899#issuecomment-534380034) and [here](https://github.com/apache/spark/pull/25899#issuecomment-534069194) for some measurements

Closes #25899 from cozos/master.

Lead-authored-by: Arwin Tio <Arwin.tio@adroll.com>
Co-authored-by: Arwin Tio <arwin.tio@hotmail.com>
Co-authored-by: Arwin Tio <arwin.tio@adroll.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-02-17 09:30:35 -06:00
Maxim Gekk 06217cfded [SPARK-30793][SQL] Fix truncations of timestamps before the epoch to minutes and seconds
### What changes were proposed in this pull request?
In the PR, I propose to replace `%` by `Math.floorMod` in `DateTimeUtils.truncTimestamp` for the `SECOND` and `MINUTE` levels.

### Why are the changes needed?
This fixes the issue of incorrect truncation of timestamps before the epoch `1970-01-01T00:00:00.000000Z` to the `SECOND` and `MINUTE` levels. For example, timestamps after the epoch are truncated by cutting off the rest part of the timestamp:
```sql
spark-sql> select date_trunc('SECOND', '2020-02-11 00:01:02.123');
2020-02-11 00:01:02
```
but seconds in the truncated timestamp before the epoch are increased by 1:
```sql
spark-sql> select date_trunc('SECOND', '1960-02-11 00:01:02.123');
1960-02-11 00:01:03
```

### Does this PR introduce any user-facing change?
Yes. After the changes, the example above outputs correct result:
```sql
spark-sql> select date_trunc('SECOND', '1960-02-11 00:01:02.123');
1960-02-11 00:01:02
```

### How was this patch tested?
Added new tests to `DateFunctionsSuite`.

Closes #27543 from MaxGekk/fix-second-minute-truc.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 22:51:56 +08:00
Yuanjian Li ab186e3659 [SPARK-25829][SQL] Add config spark.sql.legacy.allowDuplicatedMapKeys and change the default behavior
### What changes were proposed in this pull request?
This is a follow-up for #23124, add a new config `spark.sql.legacy.allowDuplicatedMapKeys` to control the behavior of removing duplicated map keys in build-in functions. With the default value `false`, Spark will throw a RuntimeException while duplicated keys are found.

### Why are the changes needed?
Prevent silent behavior changes.

### Does this PR introduce any user-facing change?
Yes, new config added and the default behavior for duplicated map keys changed to RuntimeException thrown.

### How was this patch tested?
Modify existing UT.

Closes #27478 from xuanyuanking/SPARK-25892-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 22:06:58 +08:00
Maxim Gekk 9107f77f15 [SPARK-30843][SQL] Fix getting of time components before 1582 year
### What changes were proposed in this pull request?

1. Rewrite DateTimeUtils methods `getHours()`, `getMinutes()`, `getSeconds()`, `getSecondsWithFraction()`, `getMilliseconds()` and `getMicroseconds()` using Java 8 time APIs. This will automatically switch the `Hour`, `Minute`, `Second` and `DatePart` expressions on Proleptic Gregorian calendar.
2. Remove unused methods and constant of DateTimeUtils - `to2001`, `YearZero `, `toYearZero` and `absoluteMicroSecond()`.
3. Remove unused value `timeZone` from `TimeZoneAwareExpression` since all expressions have been migrated to Java 8 time API, and legacy instance of `TimeZone` is not needed any more.
4. Change signatures of modified DateTimeUtils methods, and pass `ZoneId` instead of `TimeZone`. This will allow to avoid unnecessary conversions `TimeZone` -> `String` -> `ZoneId`.
5. Modify tests in `DateTimeUtilsSuite` and in `DateExpressionsSuite` to pass `ZoneId` instead of `TimeZone`. Correct the tests, to pass tested zone id instead of None.

### Why are the changes needed?
The changes fix the issue of wrong results returned by the `hour()`, `minute()`, `second()`, `date_part('millisecond', ...)` and `date_part('microsecond', ....)`, see example in [SPARK-30843](https://issues.apache.org/jira/browse/SPARK-30843).

### Does this PR introduce any user-facing change?
Yes. After the changes, the results of examples from SPARK-30843:
```sql
spark-sql> select hour(timestamp '0010-01-01 00:00:00');
0
spark-sql> select minute(timestamp '0010-01-01 00:00:00');
0
spark-sql> select second(timestamp '0010-01-01 00:00:00');
0
spark-sql> select date_part('milliseconds', timestamp '0010-01-01 00:00:00');
0.000
spark-sql> select date_part('microseconds', timestamp '0010-01-01 00:00:00');
0
```

### How was this patch tested?
- By existing test suites `DateTimeUtilsSuite`, `DateExpressionsSuite` and `DateFunctionsSuite`.
- Add new tests to `DateExpressionsSuite` and `DateTimeUtilsSuite` for 10 year, like:
```scala
  input = date(10, 1, 1, 0, 0, 0, 0, zonePST)
  assert(getHours(input, zonePST) === 0)
```
- Re-run `DateTimeBenchmark` using Amazon EC2.

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

Closes #27596 from MaxGekk/localtimestamp-greg-cal.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Ubuntu <ubuntu@ip-172-31-1-30.us-west-2.compute.internal>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 13:59:21 +08:00
Wenchen Fan ab07c6300c [SPARK-30799][SQL] "spark_catalog.t" should not be resolved to temp view
### What changes were proposed in this pull request?

No v2 command supports temp views and the `ResolveCatalogs`/`ResolveSessionCatalog` framework is designed with this assumption.

However, `ResolveSessionCatalog` needs to fallback to v1 commands, which do support temp views (e.g. CACHE TABLE). To work around it, we add a hack in `CatalogAndIdentifier`, which does not expand the given identifier with current namespace if the catalog is session catalog.

This works fine in most cases, as temp views should take precedence over tables during lookup. So if `CatalogAndIdentifier` returns a single name "t", the v1 commands can still resolve it to temp views correctly, or resolve it to table "default.t" if temp view doesn't exist.

However, if users write `spark_catalog.t`, it shouldn't be resolved to temp views as temp views don't belong to any catalog. `CatalogAndIdentifier` can't distinguish between `spark_catalog.t` and `t`, so the caller side may mistakenly resolve `spark_catalog.t` to a temp view.

This PR proposes to fix this issue by
1. remove the hack in `CatalogAndIdentifier`, and clearly document that this shouldn't be used to resolve temp views.
2. update `ResolveSessionCatalog` to explicitly look up temp views first before calling `CatalogAndIdentifier`, for v1 commands that support temp views.

### Why are the changes needed?

To avoid releasing a behavior that we should not support.

Removing the hack also fixes the problem we hit in https://github.com/apache/spark/pull/27532/files#diff-57b3d87be744b7d79a9beacf8e5e5eb2R937

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

yes, now it's not allowed to refer to a temp view with `spark_catalog` prefix.

### How was this patch tested?

new tests

Closes #27550 from cloud-fan/ns.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 12:07:46 +08:00
Maxim Gekk 8b73b92aad [SPARK-30826][SQL] Respect reference case in StringStartsWith pushed down to parquet
### What changes were proposed in this pull request?
In the PR, I propose to convert the attribute name of `StringStartsWith` pushed down to the Parquet datasource to column reference via the `nameToParquetField` map. Similar conversions are performed for other source filters pushed down to parquet.

### Why are the changes needed?
This fixes the bug described in [SPARK-30826](https://issues.apache.org/jira/browse/SPARK-30826). The query from an external table:
```sql
CREATE TABLE t1 (col STRING)
USING parquet
OPTIONS (path '$path')
```
created on top of written parquet files by `Seq("42").toDF("COL").write.parquet(path)` returns wrong empty result:
```scala
spark.sql("SELECT * FROM t1 WHERE col LIKE '4%'").show
+---+
|col|
+---+
+---+
```

### Does this PR introduce any user-facing change?
Yes. After the changes the result is correct for the example above:
```scala
spark.sql("SELECT * FROM t1 WHERE col LIKE '4%'").show
+---+
|col|
+---+
| 42|
+---+
```

### How was this patch tested?
Added a test to `ParquetFilterSuite`

Closes #27574 from MaxGekk/parquet-StringStartsWith-case-sens.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-15 19:49:58 +08:00
DB Tsai d0f9614760 [SPARK-30289][SQL] Partitioned by Nested Column for InMemoryTable
### What changes were proposed in this pull request?
1. `InMemoryTable` was flatting the nested columns, and then the flatten columns was used to look up the indices which is not correct.

This PR implements partitioned by nested column for `InMemoryTable`.

### Why are the changes needed?

This PR implements partitioned by nested column for `InMemoryTable`, so we can test this features in DSv2

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

No.

### How was this patch tested?

Existing unit tests and new tests.

Closes #26929 from dbtsai/addTests.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-02-14 21:46:01 +00:00
Maxim Gekk 7137a6d065 [SPARK-30766][SQL] Fix the timestamp truncation to the HOUR and DAY levels
### What changes were proposed in this pull request?
In the PR, I propose to use Java 8 time API in timestamp truncations to the levels of `HOUR` and `DAY`. The problem is in the usage of `timeZone.getOffset(millis)` in days/hours truncations where the combined calendar (Julian + Gregorian) is used underneath.

### Why are the changes needed?
The change fix wrong truncations. For example, the following truncation to hours should print `0010-01-01 01:00:00` but it outputs wrong timestamp:
```scala
Seq("0010-01-01 01:02:03.123456").toDF()
    .select($"value".cast("timestamp").as("ts"))
    .select(date_trunc("HOUR", $"ts").cast("string"))
    .show(false)
+------------------------------------+
|CAST(date_trunc(HOUR, ts) AS STRING)|
+------------------------------------+
|0010-01-01 01:30:17                 |
+------------------------------------+
```

### Does this PR introduce any user-facing change?
Yes. After the changes, the result of the example above is:
```scala
+------------------------------------+
|CAST(date_trunc(HOUR, ts) AS STRING)|
+------------------------------------+
|0010-01-01 01:00:00                 |
+------------------------------------+
```

### How was this patch tested?
- Added new test to `DateFunctionsSuite`
- By `DateExpressionsSuite` and `DateTimeUtilsSuite`

Closes #27512 from MaxGekk/fix-trunc-old-timestamp.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 22:16:57 +08:00
HyukjinKwon 2a270a731a [SPARK-30810][SQL] Parses and convert a CSV Dataset having different column from 'value' in csv(dataset) API
### What changes were proposed in this pull request?

This PR fixes `DataFrameReader.csv(dataset: Dataset[String])` API to take a `Dataset[String]` originated from a column name different from `value`. This is a long-standing bug started from the very first place.

`CSVUtils.filterCommentAndEmpty` assumed the `Dataset[String]` to be originated with `value` column. This PR changes to use the first column name in the schema.

### Why are the changes needed?

For  `DataFrameReader.csv(dataset: Dataset[String])` to support any `Dataset[String]` as the signature indicates.

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

```scala
val ds = spark.range(2).selectExpr("concat('a,b,', id) AS text").as[String]
spark.read.option("header", true).option("inferSchema", true).csv(ds).show()
```

Before:

```
org.apache.spark.sql.AnalysisException: cannot resolve '`value`' given input columns: [text];;
'Filter (length(trim('value, None)) > 0)
+- Project [concat(a,b,, cast(id#0L as string)) AS text#2]
   +- Range (0, 2, step=1, splits=Some(2))
```

After:

```
+---+---+---+
|  a|  b|  0|
+---+---+---+
|  a|  b|  1|
+---+---+---+
```

### How was this patch tested?

Unittest was added.

Closes #27561 from HyukjinKwon/SPARK-30810.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 18:20:18 +08:00
yi.wu 99b8136a86 [SPARK-25990][SQL] ScriptTransformation should handle different data types correctly
### What changes were proposed in this pull request?

We should convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`.

### Why are the changes needed?

We may hit below exception without this change:

```
[info]    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.1.6, executor driver): java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to org.apache.hadoop.hive.common.type.HiveDecimal
[info]   	at org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector.getPrimitiveJavaObject(JavaHiveDecimalObjectInspector.java:55)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazyUtils.writePrimitiveUTF8(LazyUtils.java:321)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serialize(LazySimpleSerDe.java:292)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serializeField(LazySimpleSerDe.java:247)
[info]   	at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.doSerialize(LazySimpleSerDe.java:231)
[info]   	at org.apache.hadoop.hive.serde2.AbstractEncodingAwareSerDe.serialize(AbstractEncodingAwareSerDe.java:55)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2(ScriptTransformationExec.scala:300)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2$adapted(ScriptTransformationExec.scala:281)
[info]   	at scala.collection.Iterator.foreach(Iterator.scala:941)
[info]   	at scala.collection.Iterator.foreach$(Iterator.scala:941)
[info]   	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$1(ScriptTransformationExec.scala:281)
[info]   	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
[info]   	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1932)
[info]   	at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
```

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

No.

### How was this patch tested?

Added new test. But please note that this test returns different result between Hive1.2 and Hive2.3 due to `HiveDecimal` or `SerDe` difference(don't know the root cause yet).

Closes #27556 from Ngone51/script_transform.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 16:52:28 +08:00
maryannxue 0aed77a015 [SPARK-30801][SQL] Subqueries should not be AQE-ed if main query is not
### What changes were proposed in this pull request?
This PR makes sure AQE is either enabled or disabled for the entire query, including the main query and all subqueries.
Currently there are unsupported queries by AQE, e.g., queries that contain DPP filters. We need to make sure that if the main query is unsupported, none of the sub-queries should apply AQE, otherwise it can lead to performance regressions due to missed opportunity of sub-query reuse.

### Why are the changes needed?
To get rid of potential perf regressions when AQE is turned on.

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

### How was this patch tested?
Updated DynamicPartitionPruningSuite:
1. Removed the existing workaround `withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")`
2. Added `DynamicPartitionPruningSuiteAEOn` and `DynamicPartitionPruningSuiteAEOff` to enable testing this suite with AQE on and off options
3. Added a check in `checkPartitionPruningPredicate` to verify that the subqueries are always in sync with the main query in terms of whether AQE is applied.

Closes #27554 from maryannxue/spark-30801.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-14 11:20:55 +08:00
Ali Afroozeh e2d3983de7 [SPARK-30798][SQL] Scope Session.active in QueryExecution
### What changes were proposed in this pull request?

This PR scopes `SparkSession.active` to prevent problems with processing queries with possibly different spark sessions (and different configs). A new method, `withActive` is introduced on `SparkSession` that restores the previous spark session after the block of code is executed.

### Why are the changes needed?
`SparkSession.active` is a thread local variable that points to the current thread's spark session. It is important to note that the `SQLConf.get` method depends on `SparkSession.active`. In the current implementation it is possible that `SparkSession.active` points to a different session which causes various problems. Most of these problems arise because part of the query processing is done using the configurations of a different session. For example, when creating a data frame using a new session, i.e., `session.sql("...")`, part of the data frame is constructed using the currently active spark session, which can be a different session from the one used later for processing the query.

### Does this PR introduce any user-facing change?
The `withActive` method is introduced on `SparkSession`.

### How was this patch tested?
Unit tests (to be added)

Closes #27387 from dbaliafroozeh/UseWithActiveSessionInQueryExecution.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-13 23:58:55 +01:00
Wenchen Fan a4ceea6868 [SPARK-30751][SQL] Combine the skewed readers into one in AQE skew join optimizations
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
This is a followup of https://github.com/apache/spark/pull/26434

This PR use one special shuffle reader for skew join, so that we only have one join after optimization. In order to do that, this PR
1. add a very general `CustomShuffledRowRDD` which support all kind of partition arrangement.
2. move the logic of coalescing shuffle partitions to a util function, and call it during skew join optimization, to totally decouple with the `ReduceNumShufflePartitions` rule. It's too complicated to interfere skew join with `ReduceNumShufflePartitions`, as you need to consider the size of split partitions which don't respect target size already.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
The current skew join optimization has a serious performance issue: the size of the query plan depends on the number and size of skewed partitions.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
no

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
existing tests

test UI manually:
![image](https://user-images.githubusercontent.com/3182036/74357390-cfb30480-4dfa-11ea-83f6-825d1b9379ca.png)

explain output
```
AdaptiveSparkPlan(isFinalPlan=true)
+- OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$403a2ed5, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap1f
   +- *(5) SortMergeJoin(skew=true) [key1#2L], [key2#6L], Inner
      :- *(3) Sort [key1#2L ASC NULLS FIRST], false, 0
      :  +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB)
      :     +- ShuffleQueryStage 0
      :        +- Exchange hashpartitioning(key1#2L, 200), true, [id=#53]
      :           +- *(1) Project [(id#0L % 2) AS key1#2L]
      :              +- *(1) Filter isnotnull((id#0L % 2))
      :                 +- *(1) Range (0, 100000, step=1, splits=6)
      +- *(4) Sort [key2#6L ASC NULLS FIRST], false, 0
         +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB)
            +- ShuffleQueryStage 1
               +- Exchange hashpartitioning(key2#6L, 200), true, [id=#64]
                  +- *(2) Project [((id#4L % 2) + 1) AS key2#6L]
                     +- *(2) Filter isnotnull(((id#4L % 2) + 1))
                        +- *(2) Range (0, 100000, step=1, splits=6)
```

Closes #27493 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-13 20:09:24 +01:00
Yuming Wang fb0e07b08c [SPARK-29231][SQL] Constraints should be inferred from cast equality constraint
### What changes were proposed in this pull request?

This PR add support infer constraints from cast equality constraint. For example:
```scala
scala> spark.sql("create table spark_29231_1(c1 bigint, c2 bigint)")
res0: org.apache.spark.sql.DataFrame = []

scala> spark.sql("create table spark_29231_2(c1 int, c2 bigint)")
res1: org.apache.spark.sql.DataFrame = []

scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain
== Physical Plan ==
*(2) Project [c1#5L, c2#6L]
+- *(2) BroadcastHashJoin [c1#5L], [cast(c1#7 as bigint)], Inner, BuildRight
   :- *(2) Project [c1#5L, c2#6L]
   :  +- *(2) Filter (isnotnull(c1#5L) AND (c1#5L = 1))
   :     +- *(2) ColumnarToRow
   :        +- FileScan parquet default.spark_29231_1[c1#5L,c2#6L] Batched: true, DataFilters: [isnotnull(c1#5L), (c1#5L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct<c1:bigint,c2:bigint>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#209]
      +- *(1) Project [c1#7]
         +- *(1) Filter isnotnull(c1#7)
            +- *(1) ColumnarToRow
               +- FileScan parquet default.spark_29231_2[c1#7] Batched: true, DataFilters: [isnotnull(c1#7)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct<c1:int>
```

After this PR:
```scala
scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain
== Physical Plan ==
*(2) Project [c1#0L, c2#1L]
+- *(2) BroadcastHashJoin [c1#0L], [cast(c1#2 as bigint)], Inner, BuildRight
   :- *(2) Project [c1#0L, c2#1L]
   :  +- *(2) Filter (isnotnull(c1#0L) AND (c1#0L = 1))
   :     +- *(2) ColumnarToRow
   :        +- FileScan parquet default.spark_29231_1[c1#0L,c2#1L] Batched: true, DataFilters: [isnotnull(c1#0L), (c1#0L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct<c1:bigint,c2:bigint>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#99]
      +- *(1) Project [c1#2]
         +- *(1) Filter ((cast(c1#2 as bigint) = 1) AND isnotnull(c1#2))
            +- *(1) ColumnarToRow
               +- FileScan parquet default.spark_29231_2[c1#2] Batched: true, DataFilters: [(cast(c1#2 as bigint) = 1), isnotnull(c1#2)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct<c1:int>
```

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #27252 from wangyum/SPARK-29231.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 22:48:27 +08:00
beliefer 04604b9899 [SPARK-30758][SQL][TESTS] Improve bracketed comments tests
### What changes were proposed in this pull request?
Although Spark SQL support bracketed comments, but `SQLQueryTestSuite` can't treat bracketed comments well and lead to generated golden files can't display bracketed comments well.
This PR will improve the treatment of bracketed comments and add three test case in `PlanParserSuite`.
Spark SQL can't support nested bracketed comments and https://github.com/apache/spark/pull/27495 used to support it.

### Why are the changes needed?
Golden files can't display well.

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

### How was this patch tested?
New UT.

Closes #27481 from beliefer/ansi-brancket-comments.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 22:06:24 +08:00
Terry Kim a6b4b914f2 [SPARK-30613][SQL] Support Hive style REPLACE COLUMNS syntax
### What changes were proposed in this pull request?

This PR proposes to support Hive-style `ALTER TABLE ... REPLACE COLUMNS ...` as described in https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Add/ReplaceColumns

The user now can do the following:
```SQL
CREATE TABLE t (col1 int, col2 int) USING Foo;
ALTER TABLE t REPLACE COLUMNS (col2 string COMMENT 'comment2', col3 int COMMENT 'comment3');
```
, which drops the existing columns `col1` and `col2`, and add new columns `col2` and `col3`.

### Why are the changes needed?

This is a new DDL statement. Spark currently supports the Hive-style `ALTER TABLE ... CHANGE COLUMN ...`, so this new addition can be useful.

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

Yes, adding a new DDL statement.

### How was this patch tested?

More tests to be added.

Closes #27482 from imback82/replace_cols.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 20:13:36 +08:00
maryannxue 453d5261b2 [SPARK-30528][SQL] Turn off DPP subquery duplication by default
### What changes were proposed in this pull request?
This PR adds a config for Dynamic Partition Pruning subquery duplication and turns it off by default due to its potential performance regression.
When planning a DPP filter, it seeks to reuse the broadcast exchange relation if the corresponding join is a BHJ with the filter relation being on the build side, otherwise it will either opt out or plan the filter as an un-reusable subquery duplication based on the cost estimate. However, the cost estimate is not accurate and only takes into account the table scan overhead, thus adding an un-reusable subquery duplication DPP filter can sometimes cause perf regression.
This PR turns off the subquery duplication DPP filter by:
1. adding a config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly` and setting it `true` by default.
2. removing the existing meaningless config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcast` since we always want to reuse broadcast results if possible.

### Why are the changes needed?
This is to fix a potential performance regression caused by DPP.

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

### How was this patch tested?
Updated DynamicPartitionPruningSuite to test the new configuration.

Closes #27551 from maryannxue/spark-30528.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 19:32:38 +08:00
iRakson 926e3a1efe [SPARK-30790] The dataType of map() should be map<null,null>
### What changes were proposed in this pull request?

`spark.sql("select map()")` returns {}.

After these changes it will return map<null,null>

### Why are the changes needed?
After changes introduced due to #27521, it is important to maintain consistency while using map().

### Does this PR introduce any user-facing change?
Yes. Now map() will give map<null,null> instead of {}.

### How was this patch tested?
UT added. Migration guide updated as well

Closes #27542 from iRakson/SPARK-30790.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 12:23:40 +08:00
Thomas Graves 496f6ac860 [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes
### What changes were proposed in this pull request?

This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile.  Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile.

The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles.

I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents.

There are various places in the code that use executor "slots" for things.  The ResourceProfile adds functionality to keep that calculation in it.   This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there.
This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things.

This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier.   That full set of changes will come with the scheduler PR that will be after this one.

The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue.

The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well.

### Why are the changes needed?

Needed for stage level scheduling feature.

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

No user facing api changes added here.

### How was this patch tested?

Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases.

Closes #27313 from tgravescs/SPARK-29148.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-12 16:45:42 -06:00
Liang-Chi Hsieh 5b76367a9d [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table
### What changes were proposed in this pull request?

This is a follow-up to the PR #26956. In #26956, the patch proposed to preserve path permission when truncating table. When setting up original ACLs, we need to set user/group/other permission as ACL entries too, otherwise if the path doesn't have default user/group/other ACL entries, ACL API will complain an error `Invalid ACL: the user, group and other entries are required.`.

 In short this change makes sure:

1. Permissions for user/group/other are always kept into ACLs to work with ACL API.
2. Other custom ACLs are still kept after TRUNCATE TABLE (#26956 did this).

### Why are the changes needed?

Without this fix, `TRUNCATE TABLE` will get an error when setting up ACLs if there is no default default user/group/other ACL entries.

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

No

### How was this patch tested?

Update unit test.

Manual test on dev Spark cluster.

Set ACLs for a table path without default user/group/other ACL entries:
```
hdfs dfs -setfacl --set 'user:liangchi:rwx,user::rwx,group::r--,other::r--' /user/hive/warehouse/test.db/test_truncate_table

hdfs dfs -getfacl /user/hive/warehouse/test.db/test_truncate_table
# file: /user/hive/warehouse/test.db/test_truncate_table
# owner: liangchi
# group: supergroup
user::rwx
user:liangchi:rwx
group::r--
mask::rwx
other::r--
```
Then run `sql("truncate table test.test_truncate_table")`, it works by normally truncating the table and preserve ACLs.

Closes #27548 from viirya/fix-truncate-table-permission.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-12 14:27:18 -08:00
Maxim Gekk aa0d13683c [SPARK-30760][SQL] Port millisToDays and daysToMillis on Java 8 time API
### What changes were proposed in this pull request?
In the PR, I propose to rewrite the `millisToDays` and `daysToMillis` of `DateTimeUtils` using Java 8 time API.

I removed `getOffsetFromLocalMillis` from `DateTimeUtils` because it is a private methods, and is not used anymore in Spark SQL.

### Why are the changes needed?
New implementation is based on Proleptic Gregorian calendar which has been already used by other date-time functions. This changes make `millisToDays` and `daysToMillis` consistent to rest Spark SQL API related to date & time operations.

### Does this PR introduce any user-facing change?
Yes, this might effect behavior for old dates before 1582 year.

### How was this patch tested?
By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, DateExpressionsSuite`, `SQLQuerySuite` and `HiveResultSuite`.

Closes #27494 from MaxGekk/millis-2-days-java8-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 02:31:48 +08:00
Eric Wu 5919bd3b8d [SPARK-30651][SQL] Add detailed information for Aggregate operators in EXPLAIN FORMATTED
### What changes were proposed in this pull request?
Currently `EXPLAIN FORMATTED` only report input attributes of HashAggregate/ObjectHashAggregate/SortAggregate, while `EXPLAIN EXTENDED` provides more information of Keys, Functions, etc. This PR enhanced `EXPLAIN FORMATTED` to sync with original explain behavior.

### Why are the changes needed?
The newly added `EXPLAIN FORMATTED` got less information comparing to the original `EXPLAIN EXTENDED`

### Does this PR introduce any user-facing change?
Yes, taking HashAggregate explain result as example.

**SQL**
```
EXPLAIN FORMATTED
  SELECT
    COUNT(val) + SUM(key) as TOTAL,
    COUNT(key) FILTER (WHERE val > 1)
  FROM explain_temp1;
```

**EXPLAIN EXTENDED**
```
== Physical Plan ==
*(2) HashAggregate(keys=[], functions=[count(val#6), sum(cast(key#5 as bigint)), count(key#5)], output=[TOTAL#62L, count(key) FILTER (WHERE (val > 1))#71L])
+- Exchange SinglePartition, true, [id=#89]
   +- HashAggregate(keys=[], functions=[partial_count(val#6), partial_sum(cast(key#5 as bigint)), partial_count(key#5) FILTER (WHERE (val#6 > 1))], output=[count#75L, sum#76L, count#77L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.explain_temp1[key#5,val#6] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/XXX/spark-dev/spark/spark-warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,val:int>
```

**EXPLAIN FORMATTED - BEFORE**
```
== Physical Plan ==
* HashAggregate (5)
+- Exchange (4)
   +- HashAggregate (3)
      +- * ColumnarToRow (2)
         +- Scan parquet default.explain_temp1 (1)

...
...
(5) HashAggregate [codegen id : 2]
Input: [count#91L, sum#92L, count#93L]
...
...
```

**EXPLAIN FORMATTED - AFTER**
```
== Physical Plan ==
* HashAggregate (5)
+- Exchange (4)
   +- HashAggregate (3)
      +- * ColumnarToRow (2)
         +- Scan parquet default.explain_temp1 (1)

...
...
(5) HashAggregate [codegen id : 2]
Input: [count#91L, sum#92L, count#93L]
Keys: []
Functions: [count(val#6), sum(cast(key#5 as bigint)), count(key#5)]
Results: [(count(val#6)#84L + sum(cast(key#5 as bigint))#85L) AS TOTAL#78L, count(key#5)#86L AS count(key) FILTER (WHERE (val > 1))#87L]
Output: [TOTAL#78L, count(key) FILTER (WHERE (val > 1))#87L]
...
...
```

### How was this patch tested?
Three tests added in explain.sql for HashAggregate/ObjectHashAggregate/SortAggregate.

Closes #27368 from Eric5553/ExplainFormattedAgg.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-13 02:00:23 +08:00
Maxim Gekk 61b1e608f0 [SPARK-30759][SQL][TESTS][FOLLOWUP] Check cache initialization in StringRegexExpression
### What changes were proposed in this pull request?
Added new test to `RegexpExpressionsSuite` which checks that `cache` of compiled pattern is set when the `right` expression (pattern in `LIKE`) is a foldable expression.

### Why are the changes needed?
To be sure that `cache` in `StringRegexExpression` is initialized for foldable patterns.

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

### How was this patch tested?
By running the added test in `RegexpExpressionsSuite`.

Closes #27547 from MaxGekk/regexp-cache-test.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 23:50:34 +08:00
Maxim Gekk c1986204e5 [SPARK-30788][SQL] Support SimpleDateFormat and FastDateFormat as legacy date/timestamp formatters
### What changes were proposed in this pull request?
In the PR, I propose to add legacy date/timestamp formatters based on `SimpleDateFormat` and `FastDateFormat`:
- `LegacyFastTimestampFormatter` - uses `FastDateFormat` and supports parsing/formatting in microsecond precision. The code was borrowed from Spark 2.4, see https://github.com/apache/spark/pull/26507 & https://github.com/apache/spark/pull/26582
- `LegacySimpleTimestampFormatter` uses `SimpleDateFormat`, and support the `lenient` mode. When the `lenient` parameter is set to `false`, the parser become much stronger in checking its input.

### Why are the changes needed?
Spark 2.4.x uses the following parsers for parsing/formatting date/timestamp strings:
- `DateTimeFormat` in CSV/JSON datasource
- `SimpleDateFormat` - is used in JDBC datasource, in partitions parsing.
- `SimpleDateFormat` in strong mode (`lenient = false`), see https://github.com/apache/spark/blob/branch-2.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala#L124. It is used by the `date_format`, `from_unixtime`, `unix_timestamp` and `to_unix_timestamp` functions.

The PR aims to make Spark 3.0 compatible with Spark 2.4.x in all those cases when `spark.sql.legacy.timeParser.enabled` is set to `true`.

### Does this PR introduce any user-facing change?
This shouldn't change behavior with default settings. If `spark.sql.legacy.timeParser.enabled` is set to `true`, users should observe behavior of Spark 2.4.

### How was this patch tested?
- Modified tests in `DateExpressionsSuite` to check the legacy parser - `SimpleDateFormat`.
- Added `CSVLegacyTimeParserSuite` and `JsonLegacyTimeParserSuite` to run `CSVSuite` and `JsonSuite` with the legacy parser - `FastDateFormat`.

Closes #27524 from MaxGekk/timestamp-formatter-legacy-fallback.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 20:12:38 +08:00
beliefer f5026b1ba7 [SPARK-30763][SQL] Fix java.lang.IndexOutOfBoundsException No group 1 for regexp_extract
### What changes were proposed in this pull request?
The current implement of `regexp_extract` will throws a unprocessed exception show below:

`SELECT regexp_extract('1a 2b 14m', 'd+')`
```
java.lang.IndexOutOfBoundsException: No group 1
[info] at java.util.regex.Matcher.group(Matcher.java:538)
[info] at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
[info] at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
[info] at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729)
```
I think should treat this exception well.

### Why are the changes needed?
Fix a bug `java.lang.IndexOutOfBoundsException No group 1 `

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

### How was this patch tested?
New UT

Closes #27508 from beliefer/fix-regexp_extract-bug.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-12 14:49:22 +08:00
Kris Mok b4769998ef [SPARK-30795][SQL] Spark SQL codegen's code() interpolator should treat escapes like Scala's StringContext.s()
### What changes were proposed in this pull request?

This PR proposes to make the `code` string interpolator treat escapes the same way as Scala's builtin `StringContext.s()` string interpolator. This will remove the need for an ugly workaround in `Like` expression's codegen.

### Why are the changes needed?

The `code()` string interpolator in Spark SQL's code generator should treat escapes like Scala's builtin `StringContext.s()` interpolator, i.e. it should treat escapes in the code parts, and should not treat escapes in the input arguments.

For example,
```scala
val arg = "This is an argument."
val str = s"This is string part 1. $arg This is string part 2."
val code = code"This is string part 1. $arg This is string part 2."
assert(code.toString == str)
```
We should expect the `code()` interpolator to produce the same result as the `StringContext.s()` interpolator, where only escapes in the string parts should be treated, while the args should be kept verbatim.

But in the current implementation, due to the eager folding of code parts and literal input args, the escape treatment is incorrectly done on both code parts and literal args.
That causes a problem when an arg contains escape sequences and wants to preserve that in the final produced code string. For example, in `Like` expression's codegen, there's an ugly workaround for this bug:
```scala
      // We need double escape to avoid org.codehaus.commons.compiler.CompileException.
      // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'.
      // '\"' will cause exception 'Line break in literal not allowed'.
      val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') {
        s"""\\\\\\$escapeChar"""
      } else {
        escapeChar
      }
```

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

No.

### How was this patch tested?

Added a new unit test case in `CodeBlockSuite`.

Closes #27544 from rednaxelafx/fix-code-string-interpolator.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 15:19:16 +09:00
herman b25359cca3 [SPARK-30780][SQL] Empty LocalTableScan should use RDD without partitions
### What changes were proposed in this pull request?
This is a small follow-up for https://github.com/apache/spark/pull/27400. This PR makes an empty `LocalTableScanExec` return an `RDD` without partitions.

### Why are the changes needed?
It is a bit unexpected that the RDD contains partitions if there is not work to do. It also can save a bit of work when this is used in a more complex plan.

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

### How was this patch tested?
Added test to `SparkPlanSuite`.

Closes #27530 from hvanhovell/SPARK-30780.

Authored-by: herman <herman@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-12 10:48:29 +09:00
Maxim Gekk 45db48e2d2 Revert "[SPARK-30625][SQL] Support escape as third parameter of the like function
### What changes were proposed in this pull request?

In the PR, I propose to revert the commit 8aebc80e0e.

### Why are the changes needed?
See the concerns https://github.com/apache/spark/pull/27355#issuecomment-584344438

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

### How was this patch tested?
By existing test suites.

Closes #27531 from MaxGekk/revert-like-3-args.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-11 10:15:34 -08:00
HyukjinKwon 99bd59fe29 [SPARK-29462][SQL][DOCS] Add some more context and details in 'spark.sql.defaultUrlStreamHandlerFactory.enabled' documentation
### What changes were proposed in this pull request?

This PR adds some more information and context to `spark.sql.defaultUrlStreamHandlerFactory.enabled`.

### Why are the changes needed?

It is a bit difficult to understand the documentation of `spark.sql.defaultUrlStreamHandlerFactory.enabled`.

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

Nope, internal doc only fix.

### How was this patch tested?

Nope. I only tested linter.

Closes #27541 from HyukjinKwon/SPARK-29462-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-11 09:55:02 -08:00
Maxim Gekk dc66d57e98 [SPARK-30754][SQL] Reuse results of floorDiv in calculations of floorMod in DateTimeUtils
### What changes were proposed in this pull request?
In the case of back-to-back calculation of `floorDiv` and `floorMod` with the same arguments, the result of `foorDiv` can be reused in calculation of `floorMod`. The `floorMod` method is defined as the following in Java standard library:
```java
    public static int floorMod(int x, int y) {
        int r = x - floorDiv(x, y) * y;
        return r;
    }
```
If `floorDiv(x, y)` has been already calculated, it can be reused in `x - floorDiv(x, y) * y`.

I propose to modify 2 places in `DateTimeUtils`:
1. `microsToInstant` which is widely used in many date-time functions. `Math.floorMod(us, MICROS_PER_SECOND)` is just replaced by its definition from Java Math library.
2. `truncDate`: `Math.floorMod(oldYear, divider) == 0` is replaced by `Math.floorDiv(oldYear, divider) * divider == oldYear` where `floorDiv(...) * divider` is pre-calculated.

### Why are the changes needed?
This reduces the number of arithmetic operations, and can slightly improve performance of date-time functions.

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

### How was this patch tested?
By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`.

Closes #27491 from MaxGekk/opt-microsToInstant.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-02-11 09:07:40 -06:00
fuwhu f1d0dce484 [MINOR][DOC] Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions
### What changes were proposed in this pull request?
Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions.

### Why are the changes needed?
To describe these two classes.

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

### How was this patch tested?
no

Closes #27535 from fuwhu/SPARK-15616-FOLLOW-UP.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-11 22:16:44 +08:00
HyukjinKwon 0045be766b [SPARK-29462][SQL] The data type of "array()" should be array<null>
### What changes were proposed in this pull request?

This brings https://github.com/apache/spark/pull/26324 back. It was reverted basically because, firstly Hive compatibility, and the lack of investigations in other DBMSes and ANSI.

- In case of PostgreSQL seems coercing NULL literal to TEXT type.
- Presto seems coercing `array() + array(1)` -> array of int.
- Hive seems  `array() + array(1)` -> array of strings

 Given that, the design choices have been differently made for some reasons. If we pick one of both, seems coercing to array of int makes much more sense.

Another investigation was made offline internally. Seems ANSI SQL 2011, section 6.5 "<contextually typed value specification>" states:

> If ES is specified, then let ET be the element type determined by the context in which ES appears. The declared type DT of ES is Case:
>
> a) If ES simply contains ARRAY, then ET ARRAY[0].
>
> b) If ES simply contains MULTISET, then ET MULTISET.
>
> ES is effectively replaced by CAST ( ES AS DT )

From reading other related context, doing it to `NullType`. Given the investigation made, choosing to `null` seems correct, and we have a reference Presto now. Therefore, this PR proposes to bring it back.

### Why are the changes needed?
When empty array is created, it should be declared as array<null>.

### Does this PR introduce any user-facing change?
Yes, `array()` creates `array<null>`. Now `array(1) + array()` can correctly create `array(1)` instead of `array("1")`.

### How was this patch tested?
Tested manually

Closes #27521 from HyukjinKwon/SPARK-29462.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Aman Omer <amanomer1996@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-11 17:22:08 +09:00
Shixiong Zhu e2ebca733c [SPARK-30779][SS] Fix some API issues found when reviewing Structured Streaming API docs
### What changes were proposed in this pull request?

- Fix the scope of `Logging.initializeForcefully` so that it doesn't appear in subclasses' public methods. Right now, `sc.initializeForcefully(false, false)` is allowed to called.
- Don't show classes under `org.apache.spark.internal` package in API docs.
- Add missing `since` annotation.
- Fix the scope of `ArrowUtils` to remove it from the API docs.

### Why are the changes needed?

Avoid leaking APIs unintentionally in Spark 3.0.0.

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

No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?

Manually generated the API docs and verified the above issues have been fixed.

Closes #27528 from zsxwing/audit-ss-apis.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-10 14:26:14 -08:00
Yuanjian Li a6b91d2bf7 [SPARK-30556][SQL][FOLLOWUP] Reset the status changed in SQLExecution.withThreadLocalCaptured
### What changes were proposed in this pull request?
Follow up for #27267, reset the status changed in SQLExecution.withThreadLocalCaptured.

### Why are the changes needed?
For code safety.

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

### How was this patch tested?
Existing UT.

Closes #27516 from xuanyuanking/SPARK-30556-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-10 22:16:25 +01:00
Maxim Gekk 3c1c9b48fc [SPARK-30759][SQL] Initialize cache for foldable patterns in StringRegexExpression
### What changes were proposed in this pull request?
In the PR, I propose to fix `cache` initialization in `StringRegexExpression` by changing `case Literal(value: String, StringType)` to `case p: Expression if p.foldable`

### Why are the changes needed?
Actually, the case doesn't work at all because of:
1. Literals value has type `UTF8String`
2. It doesn't work for foldable expressions like in the example:
```sql
SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*';
```
<img width="649" alt="Screen Shot 2020-02-08 at 22 45 50" src="https://user-images.githubusercontent.com/1580697/74091681-0d4a2180-4acb-11ea-8a0d-7e8c65f4214e.png">

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

### How was this patch tested?
By the `check outputs of expression examples` test from `SQLQuerySuite`.

Closes #27502 from MaxGekk/str-regexp-foldable-pattern.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-10 12:51:37 -08:00
HyukjinKwon 4439b29bd2 Revert "[SPARK-30245][SQL] Add cache for Like and RLike when pattern is not static"
### What changes were proposed in this pull request?

This reverts commit 8ce7962931. There's variable name conflicts with 8aebc80e0e (diff-39298b470865a4cbc67398a4ea11e767).

This can be cleanly ported back to branch-3.0.

### Why are the changes needed?
Performance investigation were not made enough and it's not clear if it really beneficial or now.

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

### How was this patch tested?
Jenkins tests.

Closes #27514 from HyukjinKwon/revert-cache-PR.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-10 10:56:43 -08:00
Liang-Chi Hsieh acfdb46a60 [SPARK-27946][SQL][FOLLOW-UP] Change doc and error message for SHOW CREATE TABLE
### What changes were proposed in this pull request?

This is a follow-up for #24938 to tweak error message and migration doc.

### Why are the changes needed?

Making user know workaround if SHOW CREATE TABLE doesn't work for some Hive tables.

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

No

### How was this patch tested?

Existing unit tests.

Closes #27505 from viirya/SPARK-27946-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2020-02-10 10:45:00 -08:00
Eric Wu b2011a295b [SPARK-30326][SQL] Raise exception if analyzer exceed max iterations
### What changes were proposed in this pull request?
Enhance RuleExecutor strategy to take different actions when exceeding max iterations. And raise exception if analyzer exceed max iterations.

### Why are the changes needed?
Currently, both analyzer and optimizer just log warning message if rule execution exceed max iterations. They should have different behavior. Analyzer should raise exception to indicates the plan is not fixed after max iterations, while optimizer just log warning to keep the current plan. This is more feasible after SPARK-30138 was introduced.

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

### How was this patch tested?
Add test in AnalysisSuite

Closes #26977 from Eric5553/EnhanceMaxIterations.

Authored-by: Eric Wu <492960551@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 23:41:39 +08:00
jiake 5a240603fd [SPARK-30719][SQL] Add unit test to verify the log warning print when intentionally skip AQE
### What changes were proposed in this pull request?

This is a follow up in [#27452](https://github.com/apache/spark/pull/27452).
Add a unit test to verify whether the log warning is print when intentionally skip AQE.

### Why are the changes needed?

Add unit test

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

No

### How was this patch tested?

adding unit test

Closes #27515 from JkSelf/aqeLoggingWarningTest.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 21:48:00 +08:00
Terry Kim 70e545a94d [SPARK-30757][SQL][DOC] Update the doc on TableCatalog.alterTable's behavior
### What changes were proposed in this pull request?

This PR updates the documentation on `TableCatalog.alterTable`s behavior on the order by which the requested changes are applied. It now explicitly mentions that the changes are applied in the order given.

### Why are the changes needed?

The current documentation on `TableCatalog.alterTable` doesn't mention which order the requested changes are applied. It will be useful to explicitly document this behavior so that the user can expect the behavior. For example, `REPLACE COLUMNS` needs to delete columns before adding new columns, and if the order is guaranteed by `alterTable`, it's much easier to work with the catalog API.

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

Yes, document change.

### How was this patch tested?

Not added (doc changes).

Closes #27496 from imback82/catalog_table_alter_table.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-10 19:04:49 +08:00
Kent Yao 58b9ca1e6f [SPARK-30592][SQL][FOLLOWUP] Add some round-trip test cases
### What changes were proposed in this pull request?

Add round-trip tests for CSV and JSON functions as  https://github.com/apache/spark/pull/27317#discussion_r376745135 asked.

### Why are the changes needed?

improve test coverage

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

no
### How was this patch tested?

add uts

Closes #27510 from yaooqinn/SPARK-30592-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-10 16:23:44 +09:00
Liang-Chi Hsieh 9f8172e96a Revert "[SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project
This reverts commit a0e63b61e7.

### What changes were proposed in this pull request?

This reverts the patch at #26978 based on gatorsmile's suggestion.

### Why are the changes needed?

Original patch #26978 has not considered a corner case. We may need to put more time on ensuring we can cover all cases.

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

No

### How was this patch tested?

Unit test.

Closes #27504 from viirya/revert-SPARK-29721.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-09 19:45:16 -08:00
Gengliang Wang b877aac146 [SPARK-30684 ][WEBUI][FollowUp] A new approach for SPARK-30684
### What changes were proposed in this pull request?

Simplify the changes for adding metrics description for WholeStageCodegen in https://github.com/apache/spark/pull/27405

### Why are the changes needed?

In https://github.com/apache/spark/pull/27405, the UI changes can be made without using the function `adjustPositionOfOperationName` to adjust the position of operation name and mark as an operation-name class.

I suggest we make simpler changes so that it would be easier for future development.

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

No

### How was this patch tested?

Manual test with the queries provided in https://github.com/apache/spark/pull/27405
```
sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show
sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output")
sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output")
```
![image](https://user-images.githubusercontent.com/1097932/74073629-e3f09f00-49bf-11ea-90dc-1edb5ca29e5e.png)

Closes #27490 from gengliangwang/wholeCodegenUI.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-02-09 14:18:51 -08:00
Nicholas Chammas 339c0f9a62 [SPARK-30510][SQL][DOCS] Publicly document Spark SQL configuration options
### What changes were proposed in this pull request?

This PR adds a doc builder for Spark SQL's configuration options.

Here's what the new Spark SQL config docs look like ([configuration.html.zip](https://github.com/apache/spark/files/4172109/configuration.html.zip)):

![Screen Shot 2020-02-07 at 12 13 23 PM](https://user-images.githubusercontent.com/1039369/74050007-425b5480-49a3-11ea-818c-42700c54d1fb.png)

Compare this to the [current docs](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql):

![Screen Shot 2020-02-04 at 4 55 10 PM](https://user-images.githubusercontent.com/1039369/73790828-24a5a980-476f-11ea-998c-12cd613883e8.png)

### Why are the changes needed?

There is no visibility into the various Spark SQL configs on [the config docs page](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql).

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

No, apart from new documentation.

### How was this patch tested?

I tested this manually by building the docs and reviewing them in my browser.

Closes #27459 from nchammas/SPARK-30510-spark-sql-options.

Authored-by: Nicholas Chammas <nicholas.chammas@liveramp.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-09 19:20:47 +09:00
Yuanjian Li 3db3e39f11 [SPARK-28228][SQL] Change the default behavior for name conflict in nested WITH clause
### What changes were proposed in this pull request?
This is a follow-up for #25029, in this PR we throw an AnalysisException when name conflict is detected in nested WITH clause. In this way, the config `spark.sql.legacy.ctePrecedence.enabled` should be set explicitly for the expected behavior.

### Why are the changes needed?
The original change might risky to end-users, it changes behavior silently.

### Does this PR introduce any user-facing change?
Yes, change the config `spark.sql.legacy.ctePrecedence.enabled` as optional.

### How was this patch tested?
New UT.

Closes #27454 from xuanyuanking/SPARK-28228-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-08 14:10:28 -08:00
Terry Kim a7451f44d2 [SPARK-30614][SQL] The native ALTER COLUMN syntax should change one property at a time
### What changes were proposed in this pull request?

The current ALTER COLUMN syntax allows to change multiple properties at a time:
```
ALTER TABLE table=multipartIdentifier
  (ALTER | CHANGE) COLUMN? column=multipartIdentifier
  (TYPE dataType)?
  (COMMENT comment=STRING)?
  colPosition?
```
The SQL standard (section 11.12) only allows changing one property at a time. This is also true on other recent SQL systems like [snowflake](https://docs.snowflake.net/manuals/sql-reference/sql/alter-table-column.html) and [redshift](https://docs.aws.amazon.com/redshift/latest/dg/r_ALTER_TABLE.html). (credit to cloud-fan)

This PR proposes to change ALTER COLUMN to follow SQL standard, thus allows altering only one column property at a time.

Note that ALTER COLUMN syntax being changed here is newly added in Spark 3.0, so it doesn't affect Spark 2.4 behavior.

### Why are the changes needed?

To follow SQL standard (and other recent SQL systems) behavior.

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

Yes, now the user can update the column properties only one at a time.

For example,
```
ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment'
```
should be broken into
```
ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint
ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment'
```

### How was this patch tested?

Updated existing tests.

Closes #27444 from imback82/alter_column_one_at_a_time.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-08 02:47:44 +08:00
Maxim Gekk a3e77773cf [SPARK-30752][SQL] Fix to_utc_timestamp on daylight saving day
### What changes were proposed in this pull request?
- Rewrite the `convertTz` method of `DateTimeUtils` using Java 8 time API
- Change types of `convertTz` parameters from `TimeZone` to `ZoneId`. This allows to avoid unnecessary conversions `TimeZone` -> `ZoneId` and performance regressions as a consequence.

### Why are the changes needed?
- Fixes incorrect behavior of `to_utc_timestamp` on daylight saving day. For example:
```scala
scala> df.select(to_utc_timestamp(lit("2019-11-03T12:00:00"), "Asia/Hong_Kong").as("local UTC")).show
+-------------------+
|          local UTC|
+-------------------+
|2019-11-03 03:00:00|
+-------------------+
```
but the result must be 2019-11-03 04:00:00:
<img width="1013" alt="Screen Shot 2020-02-06 at 20 09 36" src="https://user-images.githubusercontent.com/1580697/73960846-a129bb00-491c-11ea-92f5-45831cb28a62.png">

- Simplifies the code, and make it more maintainable
- Switches `convertTz` on Proleptic Gregorian calendar used by Java 8 time classes by default. That makes the function consistent to other date-time functions.

### Does this PR introduce any user-facing change?
Yes, after the changes `to_utc_timestamp` returns the correct result `2019-11-03 04:00:00`.

### How was this patch tested?
- By existing test suite `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`.
- Added `convert time zones on a daylight saving day` to DateFunctionsSuite

Closes #27474 from MaxGekk/port-convertTz-on-Java8-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-08 02:32:07 +08:00
Wenchen Fan 5a4c70b4e2 [SPARK-27986][SQL][FOLLOWUP] window aggregate function with filter predicate is not supported
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26656.

We don't support window aggregate function with filter predicate yet and we should fail explicitly.

Observable metrics has the same issue. This PR fixes it as well.

### Why are the changes needed?

If we simply ignore filter predicate when we don't support it, the result is wrong.

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

yea, fix the query result.

### How was this patch tested?

new tests

Closes #27476 from cloud-fan/filter.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-06 13:33:39 -08:00
Wenchen Fan 8ce58627eb [SPARK-30719][SQL] do not log warning if AQE is intentionally skipped and add a config to force apply
### What changes were proposed in this pull request?

Update `InsertAdaptiveSparkPlan` to not log warning if AQE is skipped intentionally.

This PR also add a config to not skip AQE.

### Why are the changes needed?

It's not a warning at all if we intentionally skip AQE.

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

no

### How was this patch tested?

run `AdaptiveQueryExecSuite` locally and verify that there is no warning logs.

Closes #27452 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-06 09:16:14 -08:00
yi.wu 368ee62a5d [SPARK-27297][DOC][FOLLOW-UP] Improve documentation for various Scala functions
### What changes were proposed in this pull request?

Add examples and parameter description for these Scala functions:

* transform
* exists
* forall
* aggregate
* zip_with
* transform_keys
* transform_values
* map_filter
* map_zip_with

### Why are the changes needed?

Better documentation for UX.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27449 from Ngone51/doc-funcs.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 20:34:29 +08:00
yi.wu 3f5b23340e [SPARK-30744][SQL] Optimize AnalyzePartitionCommand by calculating location sizes in parallel
### What changes were proposed in this pull request?

Use `CommandUtils.calculateTotalLocationSize` for `AnalyzePartitionCommand` in order to calculate location sizes in parallel.

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27471 from Ngone51/dev_calculate_in_parallel.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 20:20:44 +08:00
beliefer c8ef1dee90 [SPARK-29108][SQL][TESTS][FOLLOWUP] Comment out no use test case and add 'insert into' statement of window.sql (Part 2)
### What changes were proposed in this pull request?
When I running the `window_part2.sql` tests find it lack insert sql. Therefore, the output is empty.
I checked the postgresql and reference https://github.com/postgres/postgres/blob/master/src/test/regress/sql/window.sql
Although `window_part1.sql` and `window_part3.sql` exists the insert sql, I think should also add it into `window_part2.sql`.
Because only one case reference the table `empsalary` and it throws `AnalysisException`.
```
-- !query
select last(salary) over(order by salary range between 1000 preceding and 1000 following),
lag(salary) over(order by salary range between 1000 preceding and 1000 following),
salary from empsalary
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1);
```

So we should do four work:
1. comment out the only one case and create a new ticket.
2. Add `INSERT INTO empsalary`.

Note: window_part4.sql not use the table `empsalary`.

### Why are the changes needed?
Supplementary test data.

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

### How was this patch tested?
New test case

Closes #27439 from beliefer/add-insert-to-window.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-06 15:24:26 +09:00
Terry Kim c27a616450 [SPARK-30612][SQL] Resolve qualified column name with v2 tables
### What changes were proposed in this pull request?

This PR fixes the issue where queries with qualified columns like `SELECT t.a FROM t` would fail to resolve for v2 tables.

This PR would allow qualified column names in query as following:
```SQL
SELECT testcat.ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT ns2.tbl.foo FROM testcat.ns1.ns2.tbl
SELECT tbl.foo FROM testcat.ns1.ns2.tbl
```

### Why are the changes needed?

This is a bug because you cannot qualify column names in queries.

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

Yes, now users can qualify column names for v2 tables.

### How was this patch tested?

Added new tests.

Closes #27391 from imback82/qualified_col.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 13:54:17 +08:00
Wenchen Fan 3b26f807a0 [SPARK-30721][SQL][TESTS] Fix DataFrameAggregateSuite when enabling AQE
### What changes were proposed in this pull request?

update `DataFrameAggregateSuite` to make it pass with AQE

### Why are the changes needed?

We don't need to turn off AQE in `DataFrameAggregateSuite`

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

no

### How was this patch tested?

run `DataFrameAggregateSuite` locally with AQE on.

Closes #27451 from cloud-fan/aqe-test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 12:36:51 -08:00
Yuanjian Li 4938905a1c [SPARK-29864][SQL][FOLLOWUP] Reference the config for the old behavior in error message
### What changes were proposed in this pull request?
Follow up work for SPARK-29864, reference the config  `spark.sql.legacy.fromDayTimeString.enabled` in error message.

### Why are the changes needed?
For better usability.

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

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

Closes #27464 from xuanyuanking/SPARK-29864-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-05 11:19:42 -08:00
turbofei 6d507b4a31 [SPARK-26218][SQL][FOLLOW UP] Fix the corner case when casting float to Integer
### What changes were proposed in this pull request?
When spark.sql.ansi.enabled is true, for the statement:
```
select cast(cast(2147483648 as Float) as Integer) //result is 2147483647
```
Its result is 2147483647 and does not throw `ArithmeticException`.

The root cause is that, the below code does not work for some corner cases.
94fc0e3235/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala (L129-L141)

For example:

![image](https://user-images.githubusercontent.com/6757692/72074911-badfde80-332d-11ea-963e-2db0e43c33e8.png)

In this PR, I fix it by comparing Math.floor(x) with Int.MaxValue directly.

### Why are the changes needed?
Result corrupt.

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

### How was this patch tested?

Added Unit test.

Closes #27151 from turboFei/SPARK-26218-follow-up-int-overflow.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-05 21:24:02 +08:00
Maxim Gekk 459e757ed4 [SPARK-30668][SQL] Support SimpleDateFormat patterns in parsing timestamps/dates strings
### What changes were proposed in this pull request?
In the PR, I propose to partially revert the commit 51a6ba0181, and provide a legacy parser based on `FastDateFormat` which is compatible to `SimpleDateFormat`.

To enable the legacy parser, set `spark.sql.legacy.timeParser.enabled` to `true`.

### Why are the changes needed?
To allow users to restore old behavior in parsing timestamps/dates using `SimpleDateFormat` patterns. The main reason for restoring is `DateTimeFormatter`'s patterns are not fully compatible to `SimpleDateFormat` patterns, see https://issues.apache.org/jira/browse/SPARK-30668

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

### How was this patch tested?
- Added new test to `DateFunctionsSuite`
- Restored additional test cases in `JsonInferSchemaSuite`.

Closes #27441 from MaxGekk/support-simpledateformat.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-05 18:48:45 +08:00
HyukjinKwon 692e3ddb4e [SPARK-27870][PYTHON][FOLLOW-UP] Rename spark.sql.pandas.udf.buffer.size to spark.sql.execution.pandas.udf.buffer.size
### What changes were proposed in this pull request?

This PR renames `spark.sql.pandas.udf.buffer.size` to `spark.sql.execution.pandas.udf.buffer.size` to be more consistent with other pandas configuration prefixes, given:
-  `spark.sql.execution.pandas.arrowSafeTypeConversion`
- `spark.sql.execution.pandas.respectSessionTimeZone`
- `spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName`
- other configurations like `spark.sql.execution.arrow.*`.

### Why are the changes needed?

To make configuration names consistent.

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

No because this configuration was not released yet.

### How was this patch tested?

Existing tests should cover.

Closes #27450 from HyukjinKwon/SPARK-27870-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-05 11:38:33 +09:00
Dongjoon Hyun 898716980d Revert "[SPARK-28310][SQL] Support (FIRST_VALUE|LAST_VALUE)(expr[ (IGNORE|RESPECT) NULLS]?) syntax"
### What changes were proposed in this pull request?

This reverts commit b89c3de1a4.

### Why are the changes needed?

`FIRST_VALUE` is used only for window expression. Please see the discussion on https://github.com/apache/spark/pull/25082 .

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

Yes.

### How was this patch tested?

Pass the Jenkins.

Closes #27458 from dongjoon-hyun/SPARK-28310.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-04 17:26:46 -08:00
Liang-Chi Hsieh 7631275f97 [SPARK-25040][SQL][FOLLOWUP] Add legacy config for allowing empty strings for certain types in json parser
### What changes were proposed in this pull request?

This is a follow-up for #22787. In #22787 we disallowed empty strings for json parser except for string and binary types. This follow-up adds a legacy config for restoring previous behavior of allowing empty string.

### Why are the changes needed?

Adding a legacy config to make migration easy for Spark users.

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

Yes. If set this legacy config to true, the users can restore previous behavior prior to Spark 3.0.0.

### How was this patch tested?

Unit test.

Closes #27456 from viirya/SPARK-25040-followup.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-04 17:22:23 -08:00
Maxim Gekk f2dd082544 [SPARK-30725][SQL] Make legacy SQL configs as internal configs
### What changes were proposed in this pull request?
All legacy SQL configs are marked as internal configs. In particular, the following configs are updated as internals:
- spark.sql.legacy.sizeOfNull
- spark.sql.legacy.replaceDatabricksSparkAvro.enabled
- spark.sql.legacy.typeCoercion.datetimeToString.enabled
- spark.sql.legacy.looseUpcast
- spark.sql.legacy.arrayExistsFollowsThreeValuedLogic

### Why are the changes needed?
In general case, users shouldn't change legacy configs, so, they can be marked as internals.

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

### How was this patch tested?
Should be tested by jenkins build and run tests.

Closes #27448 from MaxGekk/legacy-internal-sql-conf.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-04 21:17:05 +08:00
maryannxue 6097b343ba [SPARK-30717][SQL] AQE subquery map should cache SubqueryExec instead of ExecSubqueryExpression
### What changes were proposed in this pull request?
This PR is to fix a potential bug in AQE where an `ExecSubqueryExpression` could be mistakenly replaced with another `ExecSubqueryExpression` with the same `ListQuery` but a different `child` expression.
This is because a ListQuery's id can only identify the ListQuery itself, not the parent expression `InSubquery`, but right now the `subqueryMap` in `InsertAdaptiveSparkPlan` uses the `ListQuery`'s id as key and the corresponding `InSubqueryExec` for the `ListQuery`'s parent expression as value. So the fix uses the corresponding `SubqueryExec` for the `ListQuery` itself as the map's value.

### Why are the changes needed?
This logical bug could potentially cause a wrong query plan, which could throw an exception related to unresolved columns.

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

### How was this patch tested?
Passed existing UTs.

Closes #27446 from maryannxue/spark-30717.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-04 12:31:44 +08:00
fuwhu 47659a0675 [SPARK-30525][SQL] HiveTableScanExec do not need to prune partitions again after pushing down to SessionCatalog for partition pruning
### What changes were proposed in this pull request?
HiveTableScanExec does not prune partitions again after SessionCatalog.listPartitionsByFilter called.

### Why are the changes needed?
In HiveTableScanExec, it will push down to hive metastore for partition pruning if spark.sql.hive.metastorePartitionPruning is true, and then it will prune the returned partitions again using partition filters, because some predicates, eg. "b like 'xyz'", are not supported in hive metastore. But now this problem is already fixed in HiveExternalCatalog.listPartitionsByFilter, the HiveExternalCatalog.listPartitionsByFilter can return exactly what we want now. So it is not necessary any more to double prune in HiveTableScanExec.

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

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

Closes #27232 from fuwhu/SPARK-30525.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-04 01:24:53 +08:00
Yuanjian Li a4912cee61
[SPARK-29543][SS][FOLLOWUP] Move spark.sql.streaming.ui.* configs to StaticSQLConf
### What changes were proposed in this pull request?
Put the configs below needed by Structured Streaming UI into StaticSQLConf:

- spark.sql.streaming.ui.enabled
- spark.sql.streaming.ui.retainedProgressUpdates
- spark.sql.streaming.ui.retainedQueries

### Why are the changes needed?
Make all SS UI configs consistent with other similar configs in usage and naming.

### Does this PR introduce any user-facing change?
Yes, add new static config `spark.sql.streaming.ui.retainedProgressUpdates`.

### How was this patch tested?
Existing UT.

Closes #27425 from xuanyuanking/SPARK-29543-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-02 23:37:13 -08:00
Burak Yavuz 2eccfd8a73 [SPARK-30697][SQL] Handle database and namespace exceptions in catalog.isView
### What changes were proposed in this pull request?

Adds NoSuchDatabaseException and NoSuchNamespaceException to the `isView` method for SessionCatalog.

### Why are the changes needed?

This method prevents specialized resolutions from kicking in within Analysis when using V2 Catalogs if the identifier is a specialized identifier.

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

No

### How was this patch tested?

Added test to DataSourceV2SessionCatalogSuite

Closes #27423 from brkyvz/isViewF.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-03 14:08:59 +08:00
Liang-Chi Hsieh 8eecc20b11 [SPARK-27946][SQL] Hive DDL to Spark DDL conversion USING "show create table"
## What changes were proposed in this pull request?

This patch adds a DDL command `SHOW CREATE TABLE AS SERDE`. It is used to generate Hive DDL for a Hive table.

For original `SHOW CREATE TABLE`, it now shows Spark DDL always. If given a Hive table, it tries to generate Spark DDL.

For Hive serde to data source conversion, this uses the existing mapping inside `HiveSerDe`. If can't find a mapping there, throws an analysis exception on unsupported serde configuration.

It is arguably that some Hive fileformat + row serde might be mapped to Spark data source, e.g., CSV. It is not included in this PR. To be conservative, it may not be supported.

For Hive serde properties, for now this doesn't save it to Spark DDL because it may not useful to keep Hive serde properties in Spark table.

## How was this patch tested?

Added test.

Closes #24938 from viirya/SPARK-27946.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-31 19:55:25 -08:00
yi.wu 82b4f753a0 [SPARK-30508][SQL] Add SparkSession.executeCommand API for external datasource
### What changes were proposed in this pull request?

This PR adds `SparkSession.executeCommand` API for external datasource to execute a random command like

```
val df = spark.executeCommand("xxxCommand", "xxxSource", "xxxOptions")
```
Note that the command doesn't execute in Spark, but inside an external execution engine depending on data source. And it will be eagerly executed after `executeCommand` called and the returned `DataFrame` will contain the output of the command(if any).

### Why are the changes needed?

This can be useful when user wants to execute some commands out of Spark. For example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, creating cores for Solr and so on(as HyukjinKwon suggested).

Previously, user needs to use an option to achieve the goal, e.g. `spark.read.format("xxxSource").option("command", "xxxCommand").load()`, which is kind of cumbersome. With this change, it can be more convenient for user to achieve the same goal.

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

Yes, new API from `SparkSession` and a new interface `ExternalCommandRunnableProvider`.

### How was this patch tested?

Added a new test suite.

Closes #27199 from Ngone51/dev-executeCommand.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-31 15:05:26 -08:00
Maxim Gekk 2d4b5eaee4 [SPARK-30676][CORE][TESTS] Eliminate warnings from deprecated constructors of java.lang.Integer and java.lang.Double
### What changes were proposed in this pull request?
- Replace `new Integer(0)` by a serializable instance in RDD.scala
- Use `.valueOf()` instead of constructors of `java.lang.Integer` and `java.lang.Double` because constructors has been deprecated, see https://docs.oracle.com/javase/9/docs/api/java/lang/Integer.html

### Why are the changes needed?
This fixes the following warnings:
1. RDD.scala:240: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
2. MutableProjectionSuite.scala:63: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
3. UDFSuite.scala:446: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
4. UDFSuite.scala:451: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.
5. HiveUserDefinedTypeSuite.scala:71: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.

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

### How was this patch tested?
- By RDDSuite, MutableProjectionSuite, UDFSuite and HiveUserDefinedTypeSuite

Closes #27399 from MaxGekk/eliminate-warning-part4.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-31 15:03:16 -06:00
Kousuke Saruta 18bc4e55ef [SPARK-30684][WEBUI] Show the descripton of metrics for WholeStageCodegen in DAG viz
### What changes were proposed in this pull request?

Added description for metrics shown in the WholeStageCodegen-node in DAG viz.

This is before the change is applied.
![before-changed](https://user-images.githubusercontent.com/4736016/73469870-5cf16480-43ca-11ea-9a13-714083508a3b.png)

And following is after change.
![after-fixing-layout](https://user-images.githubusercontent.com/4736016/73469364-983f6380-43c9-11ea-8b7e-ddab030d0270.png)

For this change, I also modify the layout of DAG viz.
Actually, I noticed  it's not enough to just added the description.
Following is without changing the layout.
![layout-is-broken](https://user-images.githubusercontent.com/4736016/73470178-cffadb00-43ca-11ea-86d7-aed109b105e6.png)

### Why are the changes needed?

Users can't understand what those metrics mean.

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

Yes. The layout is a little bit changed.

### How was this patch tested?

I confirm the result of DAG viz with following 3 operations.

`sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show`
`sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output")`
`sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output")`

Closes #27405 from sarutak/sql-dag-metrics.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 11:58:52 -08:00
Wenchen Fan 33546d637d Revert "[SPARK-30036][SQL] Fix: REPARTITION hint does not work with order by"
This reverts commit a2de20c0e6.
2020-02-01 03:02:52 +08:00
Jungtaek Lim (HeartSaVioR) 5e0faf9a3d [SPARK-29779][SPARK-30479][CORE][SQL][FOLLOWUP] Reflect review comments on post-hoc review
### What changes were proposed in this pull request?

This PR reflects review comments on post-hoc review among PRs for SPARK-29779 (#27085), SPARK-30479 (#27164). The list of review comments this PR addresses are below:

* https://github.com/apache/spark/pull/27085#discussion_r373304218
* https://github.com/apache/spark/pull/27164#discussion_r373300793
* https://github.com/apache/spark/pull/27164#discussion_r373301193
* https://github.com/apache/spark/pull/27164#discussion_r373301351

I also applied review comments to the CORE module (BasicEventFilterBuilder.scala) as well, as the review comments for SQL/core module (SQLEventFilterBuilder.scala) can be applied there as well.

### Why are the changes needed?

There're post-hoc reviews on PRs for such issues, like links in above section.

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

No

### How was this patch tested?

Existing UTs.

Closes #27414 from HeartSaVioR/SPARK-28869-SPARK-29779-SPARK-30479-FOLLOWUP-posthoc-reviews.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 10:17:07 -08:00
Tathagata Das 481e5211d2
[SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming limits
This PR solves two bugs related to streaming limits

**Bug 1 (SPARK-30658)**: Limit before a streaming aggregate (i.e. `df.limit(5).groupBy().count()`) in complete mode was not being planned as a stateful streaming limit. The planner rule planned a logical limit with a stateful streaming limit plan only if the query is in append mode. As a result, instead of allowing max 5 rows across batches, the planned streaming query was allowing 5 rows in every batch thus producing incorrect results.

**Solution**: Change the planner rule to plan the logical limit with a streaming limit plan even when the query is in complete mode if the logical limit has no stateful operator before it.

**Bug 2 (SPARK-30657)**: `LocalLimitExec` does not consume the iterator of the child plan. So if there is a limit after a stateful operator like streaming dedup in append mode (e.g. `df.dropDuplicates().limit(5)`), the state changes of streaming duplicate may not be committed (most stateful ops commit state changes only after the generated iterator is fully consumed).

**Solution**: Change the planner rule to always use a new `StreamingLocalLimitExec` which always fully consumes the iterator. This is the safest thing to do. However, this will introduce a performance regression as consuming the iterator is extra work. To minimize this performance impact, add an additional post-planner optimization rule to replace `StreamingLocalLimitExec` with `LocalLimitExec` when there is no stateful operator before the limit that could be affected by it.

No

Updated incorrect unit tests and added new ones

Closes #27373 from tdas/SPARK-30657.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-31 09:27:34 -08:00
yi.wu 5ccbb38a71 [SPARK-29938][SQL][FOLLOW-UP] Improve AlterTableAddPartitionCommand
All credit to Ngone51, Closes #27293.
### What changes were proposed in this pull request?
This PR improves `AlterTableAddPartitionCommand` by:
1. adds an internal config for partitions batch size to avoid hard code
2. reuse `InMemoryFileIndex.bulkListLeafFiles` to perform parallel file listing to improve code reuse

### Why are the changes needed?
Improve code quality.

### Does this PR introduce any user-facing change?
Yes. We renamed `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `spark.sql.parallelFileListingInCommands.enabled` as a side effect of this change.

### How was this patch tested?
Pass Jenkins.

Closes #27413 from xuanyuanking/SPARK-29938.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-01 01:03:00 +08:00
Burak Yavuz 290a528bff [SPARK-30615][SQL] Introduce Analyzer rule for V2 AlterTable column change resolution
### What changes were proposed in this pull request?

Adds an Analyzer rule to normalize the column names used in V2 AlterTable table changes. We need to handle all ColumnChange operations. We add an extra match statement for future proofing new changes that may be added. This prevents downstream consumers (e.g. catalogs) to deal about case sensitivity or check that columns exist, etc.

We also fix the behavior for ALTER TABLE CHANGE COLUMN (Hive style syntax) for adding comments to complex data types. Currently, the data type needs to be provided as part of the Hive style syntax. This assumes that the data type as changed when it may have not and the user only wants to add a comment, which fails in CheckAnalysis.

### Why are the changes needed?

Currently we do not handle case sensitivity correctly for ALTER TABLE ALTER COLUMN operations.

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

No, fixes a bug.

### How was this patch tested?

Introduced v2CommandsCaseSensitivitySuite and added a test around HiveStyle Change columns to PlanResolutionSuite

Closes #27350 from brkyvz/normalizeAlter.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 16:41:10 +08:00
herman a5c7090ffa [SPARK-30671][SQL] emptyDataFrame should use a LocalRelation
### What changes were proposed in this pull request?
This PR makes `SparkSession.emptyDataFrame` use an empty local relation instead of an empty RDD. This allows to optimizer to recognize this as an empty relation, and creates the opportunity to do some more aggressive optimizations.

### Why are the changes needed?
It allows us to optimize empty dataframes better.

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

### How was this patch tested?
Added a test case to `DataFrameSuite`.

Closes #27400 from hvanhovell/SPARK-30671.

Authored-by: herman <herman@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-31 16:14:07 +09:00
Dongjoon Hyun 05be81d69e [SPARK-30192][SQL][FOLLOWUP] Rename SINGLETON to INSTANCE
### What changes were proposed in this pull request?

This PR renames a variable `SINGLETON` to `INSTANCE`.

### Why are the changes needed?

This is a minor change for consistency with the other parts.

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

No.

### How was this patch tested?

Pass the existing tests.

Closes #27409 from dongjoon-hyun/SPARK-30192.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 22:51:51 -08:00
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
sandeep katta 5f3ec6250f [SPARK-30362][CORE] Update InputMetrics in DataSourceRDD
### What changes were proposed in this pull request?
Incase of DS v2 InputMetrics are not updated

**Before Fix**
![inputMetrics](https://user-images.githubusercontent.com/35216143/71501010-c216df00-288d-11ea-8522-fdd50b13eae1.png)

**After Fix** we can see that `Input Size / Records` is updated in the UI
![image](https://user-images.githubusercontent.com/35216143/71501000-b88d7700-288d-11ea-92fe-a727b2b79908.png)

### Why are the changes needed?
InputMetrics like bytesread and recordread should be updated

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

### How was this patch tested?
Added UT and also verified manually

Closes #27021 from sandeep-katta/dsv2inputmetrics.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-31 14:01:32 +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
Jungtaek Lim (HeartSaVioR) cbb714f67e [SPARK-29438][SS] Use partition ID of StateStoreAwareZipPartitionsRDD for determining partition ID of state store in stream-stream join
### What changes were proposed in this pull request?

Credit to uncleGen for discovering the problem and providing simple reproducer as UT. New UT in this patch is borrowed from #26156 and I'm retaining a commit from #26156 (except unnecessary part on this path) to properly give a credit.

This patch fixes the issue that partition ID could be mis-assigned when the query contains UNION and stream-stream join is placed on the right side. We assume the range of partition IDs as `(0 ~ number of shuffle partitions - 1)` for stateful operators, but when we use stream-stream join on the right side of UNION, the range of partition ID of task goes to `(number of partitions in left side, number of partitions in left side + number of shuffle partitions - 1)`, which `number of partitions in left side` can be changed in some cases (new UT points out the one of the cases).

The root reason of bug is that stream-stream join picks the partition ID from TaskContext, which wouldn't be same as partition ID from source if union is being used. Hopefully we can pick the right partition ID from source in StateStoreAwareZipPartitionsRDD - this patch leverages that partition ID.

### Why are the changes needed?

This patch will fix the broken of assumption of partition range on stateful operator, as well as fix the issue reported in JIRA issue SPARK-29438.

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

Yes, if their query is using UNION and stream-stream join is placed on the right side. They may encounter the problem to read state from checkpoint and may need to discard checkpoint to continue.

### How was this patch tested?

Added UT which fails on current master branch, and passes with this patch.

Closes #26162 from HeartSaVioR/SPARK-29438.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2020-01-30 20:21:43 -08:00
Liang-Chi Hsieh 5916c7d0d0 [SPARK-30673][SQL][TESTS] Test cases in HiveShowCreateTableSuite should create Hive table
### What changes were proposed in this pull request?

This patch makes the test cases in HiveShowCreateTableSuite create Hive table instead of data source table.

### Why are the changes needed?

Because SparkSQL now creates data source table if no provider is specified in SQL command, some test cases in HiveShowCreateTableSuite don't create Hive table, but data source table.

It is confusing and not good for the purpose of this test suite.

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

No, only test case.

### How was this patch tested?

Unit test.

Closes #27393 from viirya/SPARK-30673.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2020-01-30 13:23:58 -08:00
Wenchen Fan e5f572af06 [SPARK-30680][SQL] ResolvedNamespace does not require a namespace catalog
### What changes were proposed in this pull request?

Update `ResolvedNamespace` to accept catalog as `CatalogPlugin` not `SupportsNamespaces`.

This is extracted from https://github.com/apache/spark/pull/27345

### Why are the changes needed?

not all commands that need to resolve namespaces require a namespace catalog. For example, `SHOW TABLE` is implemented by `TableCatalog.listTables`, and is nothing to do with `SupportsNamespace`.

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

no

### How was this patch tested?

existing tests

Closes #27403 from cloud-fan/ns.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 10:34:59 -08:00
Wenchen Fan 7503e76af0 [SPARK-30622][SQL] commands should return dummy statistics
### What changes were proposed in this pull request?

override `Command.stats` to return a dummy statistics (Long.Max).

### Why are the changes needed?

Commands are eagerly executed. They will be converted to LocalRelation after the DataFrame is created. That said, the statistics of a command is useless. We should avoid unnecessary statistics calculation of command's children.

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

no

### How was this patch tested?

new test

Closes #27344 from cloud-fan/command.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 10:27:35 -08:00
Kazuaki Ishizaki b0db6231fd [SPARK-29020][FOLLOWUP][SQL] Update description of array_sort function
### What changes were proposed in this pull request?

This PR is a follow-up of #25728. #25728 introduces additional arguments to determine sort order. Thus, this function does not sort only in ascending order. However, the description was not updated.
This PR updates the description to follow the latest feature.

### Why are the changes needed?

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

No

### How was this patch tested?

Existing tests since this PR just updates description text.

Closes #27404 from kiszk/SPARK-29020-followup.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 09:41:32 -08:00
zero323 b1f81f0072 [MINOR][SQL][DOCS] Fix typos in scaladoc strings of higher order functions
### What changes were proposed in this pull request?

Fix following typos:

- tranformation -> transformation
- the boolean -> the Boolean
- signle -> single

### Why are the changes needed?

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

No

### How was this patch tested?

Scala linter.

Closes #27382 from zero323/functions-typos.

Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-29 18:42:18 -06:00
uncleGen 7173786153
[SPARK-29543][SS][UI] Structured Streaming Web UI
### What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:
   - "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries.
  - "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including `Input Rate`, `Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration`

![Screen Shot 2020-01-29 at 1 38 00 PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png)
![Screen Shot 2020-01-29 at 1 39 16 PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png)

### Why are the changes needed?

It helps users to better monitor Structured Streaming query.

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

No

### How was this patch tested?

- new added and existing UTs
- manual test

Closes #26201 from uncleGen/SPARK-29543.

Lead-authored-by: uncleGen <hustyugm@gmail.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Genmao Yu <hustyugm@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-29 13:43:51 -08:00
Takeshi Yamamuro ec1fb6b4e1 [SPARK-30234][SQL][FOLLOWUP] Add .enabled in the suffix of the ADD FILE legacy option
### What changes were proposed in this pull request?

This pr intends to rename `spark.sql.legacy.addDirectory.recursive` into `spark.sql.legacy.addDirectory.recursive.enabled`.

### Why are the changes needed?

For consistent option names.

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

No.

### How was this patch tested?

N/A

Closes #27372 from maropu/SPARK-30234-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-29 12:23:59 +09:00
Dongjoon Hyun 580c2b7e34 [SPARK-27166][SQL][FOLLOWUP] Refactor to build string once
### What changes were proposed in this pull request?

This is a follow-up for https://github.com/apache/spark/pull/24098 to refactor to build string once according to the [review comment](https://github.com/apache/spark/pull/24098#discussion_r369845234)

### Why are the changes needed?

Previously, we chose the minimal change way.
In this PR, we choose a more robust way than the previous post-step string processing.

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

No.

### How was this patch tested?

The test case is extended with more cases.

Closes #27353 from dongjoon-hyun/SPARK-27166-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-28 12:48:16 -08:00
Maxim Gekk 8aebc80e0e [SPARK-30625][SQL] Support escape as third parameter of the like function
### What changes were proposed in this pull request?
In the PR, I propose to transform the `Like` expression to `TernaryExpression`, and add third parameter `escape`. So, the `like` function will have feature parity with `LIKE ... ESCAPE` syntax supported by 187f3c1773.

### Why are the changes needed?
The `like` functions can be called with 2 or 3 parameters, and functionally equivalent to `LIKE` and `LIKE ... ESCAPE` SQL expressions.

### Does this PR introduce any user-facing change?
Yes, before `like` fails with the exception:
```sql
spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_');
Error in query: Invalid number of arguments for function like. Expected: 2; Found: 3; line 1 pos 7
```
After:
```sql
spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_');
true
```

### How was this patch tested?
- Add new example for the `like` function which is checked by `SQLQuerySuite`
- Run `RegexpExpressionsSuite` and `ExpressionParserSuite`.

Closes #27355 from MaxGekk/like-3-args.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-27 11:19:32 -08:00
Patrick Cording c5c580ba0d [SPARK-30633][SQL] Append L to seed when type is LongType
### What changes were proposed in this pull request?

Allow for using longs as seed for xxHash.

### Why are the changes needed?

Codegen fails when passing a seed to xxHash that is > 2^31.

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

No.

### How was this patch tested?

Existing tests pass. Should more be added?

Closes #27354 from patrickcording/fix_xxhash_seed_bug.

Authored-by: Patrick Cording <patrick.cording@datarobot.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-27 10:32:15 -08:00
Jungtaek Lim (HeartSaVioR) 0436b3d3f8 [SPARK-30653][INFRA][SQL] EOL character enforcement for java/scala/xml/py/R files
### What changes were proposed in this pull request?

This patch converts CR/LF into LF in 3 source files, which most files are only using LF. This patch also add rules to enforce EOL as LF for all java, scala, xml, py, R files.

### Why are the changes needed?

The majority of source code files are using LF and only three files are CR/LF. While using IDE would let us don't bother with the difference, it still has a chance to make unnecessary diff if the file is modified with the editor which doesn't handle it automatically.

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

No

### How was this patch tested?

```
grep -IUrl --color "^M" . | grep "\.java\|\.scala\|\.xml\|\.py\|\.R" | grep -v "/target/" | grep -v "/build/" | grep -v "/dist/" | grep -v "dependency-reduced-pom.xml" | grep -v ".pyc"
```

(Please note you'll need to type CTRL+V -> CTRL+M in bash shell to get `^M` because it's representing CR/LF, not a combination of `^` and `M`.)

Before the patch, the result is:

```
./sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java
./sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
```

and after the patch, the result is None.

And git shows WARNING message if EOL of any of source files in given types are modified to CR/LF, like below:

```
warning: CRLF will be replaced by LF in sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala.
The file will have its original line endings in your working directory.
```

Closes #27365 from HeartSaVioR/MINOR-remove-CRLF-in-source-codes.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-27 10:20:51 -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
Xiao Li 48f647882a [SPARK-30644][SQL][TEST] Remove query index from the golden files of SQLQueryTestSuite
### What changes were proposed in this pull request?

This PR is to remove query index from the golden files of SQLQueryTestSuite

### Why are the changes needed?

Because the SQLQueryTestSuite's golden files have the query index for each query, removal of any query statement [except the last one] will generate many unneeded difference. This will make code review harder. The number of changed lines is misleading.

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

### How was this patch tested?
N/A

Closes #27361 from gatorsmile/removeIndexNum.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-25 23:17:36 -08:00
Xiao Li d69ed9afdf Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit 1d20d13149.

Closes #27351 from gatorsmile/revertSPARK25496.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-25 21:34:12 -08:00
Liang-Chi Hsieh a0e63b61e7 [SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project
### What changes were proposed in this pull request?

This patch proposes to prune unnecessary nested fields from Generate which has no Project on top of it.

### Why are the changes needed?

In Optimizer, we can prune nested columns from Project(projectList, Generate). However, unnecessary columns could still possibly be read in Generate, if no Project on top of it. We should prune it too.

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

No

### How was this patch tested?

Unit test.

Closes #26978 from viirya/SPARK-29721.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-24 22:17:28 -08:00
Gengliang Wang ed44926117 [SPARK-30627][SQL] Disable all the V2 file sources by default
### What changes were proposed in this pull request?

Disable all the V2 file sources in Spark 3.0 by default.

### Why are the changes needed?

There are still some missing parts in the file source V2 framework:
1. It doesn't support reporting file scan metrics such as "numOutputRows"/"numFiles"/"fileSize" like `FileSourceScanExec`. This requires another patch in the data source V2 framework. Tracked by [SPARK-30362](https://issues.apache.org/jira/browse/SPARK-30362)
2. It doesn't support partition pruning with subqueries(including dynamic partition pruning) for now. Tracked by [SPARK-30628](https://issues.apache.org/jira/browse/SPARK-30628)

As we are going to code freeze on Jan 31st, this PR proposes to disable all the V2 file sources in Spark 3.0 by default.

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

No

### How was this patch tested?

Existing tests.

Closes #27348 from gengliangwang/disableFileSourceV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 21:42:43 -08:00
Xiao Li ddf83159a8 [SPARK-28962][SQL][FOLLOW-UP] Add the parameter description for the Scala function API filter
### What changes were proposed in this pull request?
This PR is a follow-up PR https://github.com/apache/spark/pull/25666 for adding the description and example for the Scala function API `filter`.

### Why are the changes needed?
It is hard to tell which parameter is the index column.

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

### How was this patch tested?
N/A

Closes #27336 from gatorsmile/spark28962.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 16:23:16 -08:00
Xiao Li 3f76bd4002 [SPARK-27083][SQL][FOLLOW-UP] Rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled
### What changes were proposed in this pull request?
This PR is to rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled

### Why are the changes needed?
Make it consistent and clear.

### Does this PR introduce any user-facing change?
N/A. This is a [new conf added in Spark 3.0](https://github.com/apache/spark/pull/23998)

### How was this patch tested?
N/A

Closes #27346 from gatorsmile/spark27083.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-23 15:34:54 -08:00
Terry Kim 4847f7380d [SPARK-30298][SQL] Respect aliases in output partitioning of projects and aggregates
### What changes were proposed in this pull request?

Currently, in the following scenario, bucket join is not utilized:
```scala
val df = (0 until 20).map(i => (i, i)).toDF("i", "j").as("df")
df.write.format("parquet").bucketBy(8, "i").saveAsTable("t")
sql("CREATE VIEW v AS SELECT * FROM t")
sql("SELECT * FROM t a JOIN v b ON a.i = b.i").explain
```
```
== Physical Plan ==
*(4) SortMergeJoin [i#13], [i#15], Inner
:- *(1) Sort [i#13 ASC NULLS FIRST], false, 0
:  +- *(1) Project [i#13, j#14]
:     +- *(1) Filter isnotnull(i#13)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
+- *(3) Sort [i#15 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i#15, 8), true, [id=#64] <----- Exchange node introduced
      +- *(2) Project [i#13 AS i#15, j#14 AS j#16]
         +- *(2) Filter isnotnull(i#13)
            +- *(2) ColumnarToRow
               +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
```
Notice that `Exchange` is present. This is because `Project` introduces aliases and `outputPartitioning` and `requiredChildDistribution` do not consider aliases while considering bucket join in `EnsureRequirements`. This PR addresses to allow this scenario.

### Why are the changes needed?

This allows bucket join to be utilized in the above example.

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

Yes, now with the fix, the `explain` out is as follows:
```
== Physical Plan ==
*(3) SortMergeJoin [i#13], [i#15], Inner
:- *(1) Sort [i#13 ASC NULLS FIRST], false, 0
:  +- *(1) Project [i#13, j#14]
:     +- *(1) Filter isnotnull(i#13)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
+- *(2) Sort [i#15 ASC NULLS FIRST], false, 0
   +- *(2) Project [i#13 AS i#15, j#14 AS j#16]
      +- *(2) Filter isnotnull(i#13)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct<i:int,j:int>, SelectedBucketsCount: 8 out of 8
```
Note that the `Exchange` is no longer present.

### How was this patch tested?

Closes #26943 from imback82/bucket_alias.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-24 08:23:07 +09:00
Kent Yao 3228d723a4 [SPARK-30603][SQL] Move RESERVED_PROPERTIES from SupportsNamespaces and TableCatalog to CatalogV2Util
### What changes were proposed in this pull request?
In this PR, I propose to move the `RESERVED_PROPERTIES `s from `SupportsNamespaces` and `TableCatalog` to `CatalogV2Util`, which can keep `RESERVED_PROPERTIES ` safe for interval usages only.

### Why are the changes needed?

 the `RESERVED_PROPERTIES` should not be changed by subclasses

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

no

### How was this patch tested?

existing uts

Closes #27318 from yaooqinn/SPARK-30603.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 13:13:25 -08:00
Wenchen Fan 976946a910 [SPARK-29947][SQL][FOLLOWUP] Fix table lookup cache
### What changes were proposed in this pull request?

Fix a bug in https://github.com/apache/spark/pull/26589 , to make this feature work.

### Why are the changes needed?

This feature doesn't work actually.

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

no

### How was this patch tested?

new test

Closes #27341 from cloud-fan/cache.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 13:02:10 -08:00
Wenchen Fan 3c8b3609a1 [SPARK-30620][SQL] avoid unnecessary serialization in AggregateExpression
### What changes were proposed in this pull request?

Expressions are very likely to be serialized and sent to executors, we should avoid unnecessary serialization overhead as much as we can.

This PR fixes `AggregateExpression`.

### Why are the changes needed?

small improvement

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

no

### How was this patch tested?

existing tests

Closes #27342 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-24 01:15:57 +08:00
Yuanjian Li 3d7359ad42 [SPARK-29175][SQL][FOLLOW-UP] Rename the config name to spark.sql.maven.additionalRemoteRepositories
### What changes were proposed in this pull request?
Rename the config added in #25849 to `spark.sql.maven.additionalRemoteRepositories`.

### Why are the changes needed?
Follow the advice in [SPARK-29175](https://issues.apache.org/jira/browse/SPARK-29175?focusedCommentId=17021586&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17021586), the new name is more clear.

### Does this PR introduce any user-facing change?
Yes, the config name changed.

### How was this patch tested?
Existing test.

Closes #27339 from xuanyuanking/SPARK-29175.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-23 08:35:32 -08:00
Maxim Gekk 705fc6ad93 [SPARK-30188][SQL][TESTS][FOLLOW-UP] Remove sorted in asserts of comparing two strings
### What changes were proposed in this pull request?
In the PR, I propose to remove sorting in the asserts of checking output of:
- expression examples,
- SQL tests in `SQLQueryTestSuite`.

### Why are the changes needed?
* Sorted `actual` and `expected` make assert output unusable. Instead of `"[true]" did not equal "[false]"`, it looks like `"[ertu]" did not equal "[aefls]"`.
* Output of expression examples should be always the same except nondeterministic expressions listed in the `ignoreSet` of the `check outputs of expression examples` test.

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

### How was this patch tested?
By running `SQLQuerySuite` via `./build/sbt "sql/test:testOnly org.apache.spark.sql.SQLQuerySuite"`.

Closes #27324 from MaxGekk/remove-sorting-in-examples-tests.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 23:01:25 +08:00
Wenchen Fan dbed4c72f3 [SPARK-30605][SQL] move defaultNamespace from SupportsNamespace to CatalogPlugin
### What changes were proposed in this pull request?

Move the `defaultNamespace` method from the interface `SupportsNamespace` to `CatalogPlugin`.

### Why are the changes needed?

While I'm implementing JDBC V2, I realize that the default namespace is very an important information. Even if you don't want to implement namespace manipulation functionalities like CREATE/DROP/ALTER namespace, you still need to report the default namespace.

The default namespace is not about functionality but a matter of correctness. If you don't know the default namespace of a catalog, it's wrong to assume it's `[]`. You may get table not found exception if you do so.

I think it's more reasonable to put the `defaultNamespace` method in the base class `CatalogPlugin`. It returns `[]` by default so won't bother implementation if they really don't have namespace concept.

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

yes, but for an unreleased API.

### How was this patch tested?

existing tests

Closes #27319 from cloud-fan/ns.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 21:56:54 +08:00
Xiao Li ffd435b57f [SPARK-27871][SQL][FOLLOW-UP] Remove the conf spark.sql.optimizer.reassignLambdaVariableID.enabled
### What changes were proposed in this pull request?
This PR is to remove the conf

### Why are the changes needed?
This rule can be excluded using spark.sql.optimizer.excludedRules without an extra conf

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

### How was this patch tested?
N/A

Closes #27334 from gatorsmile/spark27871.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 21:41:56 +08:00
Burak Yavuz db528e4fe1 [SPARK-30535][SQL] Revert "[] Migrate ALTER TABLE commands to the new framework
### What changes were proposed in this pull request?

This reverts commit b5cb9abdd5.

### Why are the changes needed?

The merged commit (#27243) was too risky for several reasons:
 1. It doesn't fix a bug
 2. It makes the resolution of the table that's going to be altered a child. We had avoided this on purpose as having an arbitrary rule change the child of AlterTable seemed risky. This change alone is a big -1 for me for this change.
 3. While the code may look cleaner, I think this approach makes certain things harder, e.g. differentiating between the Hive based Alter table CHANGE COLUMN and ALTER COLUMN syntax. Resolving and normalizing columns for ALTER COLUMN also becomes a bit harder, as we now have to check every single AlterTable command instead of just a single ALTER TABLE ALTER COLUMN statement

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

No

### How was this patch tested?

Existing unit tests

This closes #27315

Closes #27327 from brkyvz/revAlter.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-22 22:43:46 -08:00
Tathagata Das d2bca8ff70 [SPARK-30609] Allow default merge command resolution to be bypassed by DSv2 tables
### What changes were proposed in this pull request?
Skip resolving the merge expressions if the target is a DSv2 table with ACCEPT_ANY_SCHEMA capability.

### Why are the changes needed?
Some DSv2 sources may want to customize the merge resolution logic. For example, a table that can accept any schema (TableCapability.ACCEPT_ANY_SCHEMA) may want to allow certain merge queries that are blocked (that is, throws AnalysisError) by the default resolution logic. So there should be a way to completely bypass the merge resolution logic in the Analyzer.

### Does this PR introduce any user-facing change?
No, since merge itself is an unreleased feature

### How was this patch tested?
added unit test to specifically test the skipping.

Closes #27326 from tdas/SPARK-30609.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2020-01-22 19:20:25 -08:00
Ajith bbab2bb961 [SPARK-30556][SQL] Copy sparkContext.localproperties to child thread inSubqueryExec.executionContext
### What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.SubqueryExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the sub-execution thread in `org.apache.spark.sql.execution.SubqueryExec#executionContext`

### Why are the changes needed?
Local properties set via sparkContext are not available as TaskContext properties when executing  jobs and threadpools have idle threads which are reused

Explanation:
When `SubqueryExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads.
These threads are created in the `executionContext` (thread pools). Each Thread pool has a default keepAliveSeconds of 60 seconds for idle threads.
Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob`

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

### How was this patch tested?
Added UT

Closes #27267 from ajithme/subquerylocalprop.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-22 18:21:11 -08:00
Enrico Minack eccae13a5f [SPARK-30531][WEB UI] Do not render plan viz when it exists already
### What changes were proposed in this pull request?
When you save a Spark UI SQL query page to disk and then display the html file with your browser, the query plan will be rendered a second time. This change avoids rendering the plan visualization when it exists already.

This is master:
![grafik](https://user-images.githubusercontent.com/44700269/72543429-fcb8d980-3885-11ea-82aa-c0b3638847e5.png)

And with the fix:
![grafik](https://user-images.githubusercontent.com/44700269/72543641-57523580-3886-11ea-8cdf-5fb0cdffa983.png)

### Why are the changes needed?
The duplicate query plan is unexpected and redundant.

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

### How was this patch tested?
Manually tested. Testing this in a reproducible way requires a running browser or HTML rendering engine that executes the JavaScript.

Closes #27238 from EnricoMi/branch-sql-ui-duplicate-plan.

Authored-by: Enrico Minack <github@enrico.minack.dev>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-22 19:51:08 -06:00
Maxim Gekk 4ca31b470f [SPARK-30606][SQL] Fix the like function with 2 parameters
### What changes were proposed in this pull request?
In the PR, I propose to add additional constructor in the `Like` expression. The constructor can be used on applying the `like` function with 2 parameters.

### Why are the changes needed?
`FunctionRegistry` cannot find a constructor if the `like` function is applied to 2 parameters.

### Does this PR introduce any user-facing change?
Yes, before:
```sql
spark-sql> SELECT like('Spark', '_park');

Invalid arguments for function like; line 1 pos 7
org.apache.spark.sql.AnalysisException: Invalid arguments for function like; line 1 pos 7
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$7(FunctionRegistry.scala:618)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$4(FunctionRegistry.scala:602)
	at org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry.lookupFunction(FunctionRegistry.scala:121)
	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1412)
```
After:
```sql
spark-sql> SELECT like('Spark', '_park');
true
```

### How was this patch tested?
By running `check outputs of expression examples` from `SQLQuerySuite`.

Closes #27323 from MaxGekk/fix-like-func.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-22 15:40:24 -08:00
jiake 6dfaa0783f [SPARK-30549][SQL] Fix the subquery shown issue in UI When enable AQE
### What changes were proposed in this pull request?
After [PR#25316](https://github.com/apache/spark/pull/25316) fixed the dead lock issue in [PR#25308](https://github.com/apache/spark/pull/25308), the subquery metrics can not be shown in UI as following screenshot.
![image](https://user-images.githubusercontent.com/11972570/72891385-160ec980-3d4f-11ea-91fc-ccaad890f7dc.png)

 This PR fix the subquery UI shown issue by adding `SparkListenerSQLAdaptiveSQLMetricUpdates` event to update the suquery  sql metric. After with this PR, the suquery UI can show correctly as following screenshot:
![image](https://user-images.githubusercontent.com/11972570/72893610-66d4f100-3d54-11ea-93c9-f444b2f31952.png)

### Why are the changes needed?
Showing the subquery metric in UI when enable AQE

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

### How was this patch tested?
Existing UT

Closes #27260 from JkSelf/fixSubqueryUI.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-22 09:02:34 -08:00
Kent Yao 8e280cebf2 [SPARK-30592][SQL] Interval support for csv and json funtions
### What changes were proposed in this pull request?

In this PR, I'd propose to fully support interval for the CSV and JSON functions.

On one hand, CSV and JSON records consist of string values, in the cast logic, we can cast string from/to interval now, so we can make those functions support intervals easily.

Before this change we can only use this as a workaround.
```sql
SELECT cast(from_csv('1, 1 day', 'a INT, b string').b as interval)
struct<CAST(from_csv(1, 1 day).b AS INTERVAL):interval>
1 days
```

On the other hand,  we ban reading or writing intervals from CSV and JSON files. To directly read and write  with external json/csv storage, you still need explicit cast, e.g.
```scala
spark.read.schema("a string").json("a.json").selectExpr("cast(a as interval)").show
+------+
|     a|
+------+
|1 days|
+------+
```

### Why are the changes needed?

for interval's future-proofing purpose

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

yes, the `to_json`/`from_json` function can deal with intervals now. e.g.
for `from_json` there is no such use case because we do not support `a interval`
for `to_json`, we can use interval values now

#### before

 ```sql

 SELECT to_json(map('a', interval 25 month 100 day 130 minute));
Error in query: cannot resolve 'to_json(map('a', INTERVAL '2 years 1 months 100 days 2 hours 10 minutes'))' due to data type mismatch: Unable to convert column a of type interval to JSON.; line 1 pos 7;
'Project [unresolvedalias(to_json(map(a, 2 years 1 months 100 days 2 hours 10 minutes), Some(Asia/Shanghai)), None)]
+- OneRowRelation
```
#### after
```sql
SELECT to_json(map('a', interval 25 month 100 day 130 minute))
{"a":"2 years 1 months 100 days 2 hours 10 minutes"}

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

add ut

Closes #27317 from yaooqinn/SPARK-30592.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-23 00:41:46 +08:00
Wenchen Fan b8cb52a8a7 [SPARK-30555][SQL] MERGE INTO insert action should only access columns from source table
### What changes were proposed in this pull request?

when resolving the `Assignment` of insert action in MERGE INTO, only resolve with the source table, to avoid ambiguous attribute failure if there is a same-name column in the target table.

### Why are the changes needed?

The insert action is used when NOT MATCHED, so it can't access the row from the target table anyway.

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

on

### How was this patch tested?

new tests

Closes #27265 from cloud-fan/merge.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-22 21:31:11 +08:00
Kent Yao f2d71f5838 [SPARK-30591][SQL] Remove the nonstandard SET OWNER syntax for namespaces
### What changes were proposed in this pull request?

This pr removes the nonstandard `SET OWNER` syntax for namespaces and changes the owner reserved properties from `ownerName` and `ownerType` to `owner`.

### Why are the changes needed?

the `SET OWNER` syntax for namespaces is hive-specific and non-sql standard, we need a more future-proofing design before we implement user-facing changes for SQL security issues

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

no, just revert an unpublic syntax

### How was this patch tested?

modified uts

Closes #27300 from yaooqinn/SPARK-30591.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-22 16:00:05 +08:00
fuwhu cfb1706eaa [SPARK-15616][SQL] Add optimizer rule PruneHiveTablePartitions
### What changes were proposed in this pull request?
Add optimizer rule PruneHiveTablePartitions pruning hive table partitions based on filters on partition columns.
Doing so, the total size of pruned partitions may be small enough for broadcast join in JoinSelection strategy.

### Why are the changes needed?
In JoinSelection strategy, spark use the "plan.stats.sizeInBytes" to decide whether the plan is suitable for broadcast join.
Currently, "plan.stats.sizeInBytes" does not take "pruned partitions" into account, so it may miss some broadcast join and take sort-merge join instead, which will definitely impact join performance.
This PR aim at taking "pruned partitions" into account for hive table in "plan.stats.sizeInBytes" and then improve performance by using broadcast join if possible.

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

### How was this patch tested?
Added unit tests.

This is based on #25919, credits should go to lianhuiwang and advancedxy.

Closes #26805 from fuwhu/SPARK-15616.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:26:30 +08:00
yi.wu ff39c9271c [SPARK-30252][SQL] Disallow negative scale of Decimal
### What changes were proposed in this pull request?

This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes:

1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2);
2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws.

And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior.

### Why are the changes needed?

According to SQL standard,
> 4.4.2 Characteristics of numbers
An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer.

scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale.

Presto:
```
presto:default> create table t (i decimal(2, -1));
Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: <integer>, <type>
create table t (i decimal(2, -1))
```

PostgrelSQL:
```
postgres=# create table t(i decimal(2, -1));
ERROR:  NUMERIC scale -1 must be between 0 and precision 2
LINE 1: create table t(i decimal(2, -1));
                         ^
```

And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL:
```
scala> spark.sql("create table t(i decimal(2, -1))");
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28)

== SQL ==
create table t(i decimal(2, -1))
----------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
  ... 35 elided
```

However, it is still possible to create such table or `DatFrame` using Spark SQL programming API:
```
scala> val tb =
 CatalogTable(
  TableIdentifier("test", None),
  CatalogTableType.MANAGED,
  CatalogStorageFormat.empty,
  StructType(StructField("i", DecimalType(2, -1) ) :: Nil))
```
```
scala> spark.sql("SELECT 1.23E4BD")
res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)]
```
while, these two different behavior could make user confused.

On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it.
```
scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet")
19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2
	at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309)
	at org.apache.parquet.schema.Types$Builder.named(Types.java:290)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at org.apache.spark.sql.types.StructType.map(StructType.scala:99)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:109)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

So, I think it would be better to disallow negative scale totally and make behaviors above be consistent.

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

Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore.

### How was this patch tested?

Added new tests in `ExpressionParserSuite` and `DecimalSuite`;
Updated `SQLQueryTestSuite`.

Closes #26881 from Ngone51/nonnegative-scale.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:09:48 +08:00
Kent Yao af705421db [SPARK-30593][SQL] Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and no round trip
### What changes were proposed in this pull request?

This revert https://github.com/apache/spark/pull/26418, file a new ticket under  https://issues.apache.org/jira/browse/SPARK-30546 for better tracking interval behavior
### Why are the changes needed?

Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and there is no round trip

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

no, not released yet

### How was this patch tested?

existing uts

Closes #27304 from yaooqinn/SPARK-30593.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 20:51:10 +08:00
yi.wu 78df532556 [SPARK-30433][SQL][FOLLOW-UP] Optimize collect conflict plans
### What changes were proposed in this pull request?

For LogicalPlan(e.g. `MultiInstanceRelation`, `Project`, `Aggregate`, etc)  whose output doesn't inherit directly from its children, we could just stop collect on it. Because we could always replace all the lower conflict attributes with the new attributes from the new plan.

Otherwise, we should recursively collect conflict plans, like `Generate`, `Window`.

### Why are the changes needed?

Performance improvement.

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

No.

### How was this patch tested?

Pass existed tests.

Closes #27263 from Ngone51/spark_30433_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 14:23:55 +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 94284c8ecc [SPARK-30587][SQL][TESTS] Add test suites for CSV and JSON v1
### What changes were proposed in this pull request?
In the PR, I propose to make `JsonSuite` and `CSVSuite` abstract classes, and add sub-classes that check JSON/CSV datasource v1 and v2.

### Why are the changes needed?
To improve test coverage and test JSON/CSV v1 which is still supported, and can be enabled by users.

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

### How was this patch tested?
By running new test suites `JsonV1Suite` and `CSVv1Suite`.

Closes #27294 from MaxGekk/csv-json-v1-test-suites.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 11:38:05 +08:00
Kent Yao 0388b7a3ec [SPARK-30568][SQL] Invalidate interval type as a field table schema
### What changes were proposed in this pull request?

After this commit d67b98ea01, we are able to create table or alter table with interval column types if the external catalog accepts which is varying the interval type's purpose for internal usage. With d67b98ea01 's original purpose it should only work from cast logic.

Instead of adding type checker for the interval type from commands to commands to work among catalogs, It much simpler to treat interval as an invalid data type but can be identified by cast only.

### Why are the changes needed?

enhance interval internal usage purpose.

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

NO,
Additionally, this PR restores user behavior when using interval type to create/alter table schema, e.g. for hive catalog
for 2.4,
```java
Caused by: org.apache.spark.sql.catalyst.parser.ParseException:
DataType calendarinterval is not supported.(line 1, pos 0)
```
for master after  d67b98ea01
```java
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.IllegalArgumentException: Error: type expected at the position 0 of 'interval' but 'interval' is found.
  at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:862)
```
now with this pr, we restore the type checker in spark side.

### How was this patch tested?

add more ut

Closes #27277 from yaooqinn/SPARK-30568.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 11:14:26 +08:00
Kent Yao 24efa43826 [SPARK-30019][SQL] Add the owner property to v2 table
### What changes were proposed in this pull request?

Add `owner` property to v2 table, it is reversed by `TableCatalog`, indicates the table's owner.

### Why are the changes needed?

enhance ownership management of catalog API

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

yes, add 1 reserved property - `owner` , and it is not allowed to use in OPTIONS/TBLPROPERTIES anymore, only if legacy on

### How was this patch tested?

add uts

Closes #27249 from yaooqinn/SPARK-30019.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 10:37:49 +08:00
HyukjinKwon 14bc2a2162 [SPARK-30530][SQL][FOLLOW-UP] Remove unnecessary codes and fix comments accordingly in UnivocityParser
### What changes were proposed in this pull request?

This PR proposes to clean up `UnivocityParser`.

### Why are the changes needed?

It will slightly improve the performance since we don't do unnecessary computation for Array concatenations/creation.

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

No.

### How was this patch tested?

Manually ran the existing tests.

Closes #27287 from HyukjinKwon/SPARK-30530-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-21 10:20:01 +09: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
yi.wu f5b345cf3d [SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in OrcFilterSuite
### What changes were proposed in this pull request?

Explicitly set conf to let orc use DSv2 in `OrcFilterSuite` in both v1.2 and v2.3.

### Why are the changes needed?

Tests should not rely on default conf when they're going to test something intentionally, which can be fail when conf changes.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27285 from Ngone51/fix-orcfilter-test.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-20 21:42:33 +08:00
Terry Kim b5cb9abdd5 [SPARK-30535][SQL] Migrate ALTER TABLE commands to the new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the ALTER TABLE commands.

This PR also refactors ALTER TABLE logical plans such that they extend a base class `AlterTable`. Each plan now implements `def changes: Seq[TableChange]` for any table change operations.

Additionally, `UnresolvedV2Relation` and its usage is completely removed.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-29900](https://issues.apache.org/jira/browse/SPARK-29900).

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

No

### How was this patch tested?

Updated existing tests

Closes #27243 from imback82/v2commands_newframework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-20 21:33:44 +08:00
Maxim Gekk 00039cc482 [SPARK-30554][SQL] Return Iterable from FailureSafeParser.rawParser
### What changes were proposed in this pull request?
Changed signature of `rawParser` passed to `FailureSafeParser`. I propose to change return type from `Seq` to `Iterable`. I took `Iterable` to easier port the changes on Scala collections 2.13. Also, I replaced `Seq` by `Option` in CSV datasource - `UnivocityParser`, and in JSON parser exception one place in the case when specified schema is `StructType`, and JSON input is an array.

### Why are the changes needed?
`Seq` is unnecessary requirement for return type from rawParser which may not have multiple rows per input like CSV datasource.

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

### How was this patch tested?
By existing test suites `JsonSuite`, `UnivocityParserSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`, `CsvSuite`, and `CsvFunctionsSuite`.

Closes #27264 from MaxGekk/failuresafe-parser-seq.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-20 13:59:22 +09:00
Josh Rosen d50f8df929 [SPARK-30413][SQL] Avoid WrappedArray roundtrip in GenericArrayData constructor, plus related optimization in ParquetMapConverter
### What changes were proposed in this pull request?

This PR implements a tiny performance optimization for a `GenericArrayData` constructor, avoiding an unnecessary roundtrip through `WrappedArray` when the provided value is already an array of objects.

It also fixes a related performance problem in `ParquetRowConverter`.

### Why are the changes needed?

`GenericArrayData` has a `this(seqOrArray: Any)` constructor, which was originally added in #13138 for use in `RowEncoder` (where we may not know concrete types until runtime) but is also called (perhaps unintentionally) in a few other code paths.

In this constructor's existing implementation, a call to `new WrappedArray(Array[Object](""))` is dispatched to the `this(seqOrArray: Any)` constructor, where we then call `this(array.toSeq)`: this wraps the provided array into a `WrappedArray`, which is subsequently unwrapped in a `this(seq.toArray)` call. For an interactive example, see https://scastie.scala-lang.org/7jOHydbNTaGSU677FWA8nA

This PR changes the `this(seqOrArray: Any)` constructor so that it calls the primary `this(array: Array[Any])` constructor, allowing us to save a `.toSeq.toArray` call; this comes at the cost of one additional `case` in the `match` statement (but I believe this has a negligible performance impact relative to the other savings).

As code cleanup, I also reverted the JVM 1.7 workaround from #14271.

I also fixed a related performance problem in `ParquetRowConverter`: previously, this code called `ArrayBasedMapData.apply` which, in turn, called the `this(Any)` constructor for `GenericArrayData`: this PR's micro-benchmarks show that this is _significantly_ slower than calling the `this(Array[Any])` constructor (and I also observed time spent here during other Parquet scan benchmarking work). To fix this performance problem, I replaced the call to the  `ArrayBasedMapData.apply` method with direct calls to the `ArrayBasedMapData` and `GenericArrayData` constructors.

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

No.

### How was this patch tested?

I tested this by running code in a debugger and by running microbenchmarks (which I've added to a new `GenericArrayDataBenchmark` in this PR):

- With JDK8 benchmarks: this PR's changes more than double the performance of calls to the `this(Any)` constructor. Even after improvements, however, calls to the `this(Array[Any])` constructor are still ~60x faster than calls to `this(Any)` when passing a non-primitive array (thereby motivating this patch's other change in `ParquetRowConverter`).
- With JDK11 benchmarks: the changes more-or-less completely eliminate the performance penalty associated with the `this(Any)` constructor.

Closes #27088 from JoshRosen/joshrosen/GenericArrayData-optimization.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-19 19:12:19 -08:00
Maxim Gekk d4c6ec6ba7 [SPARK-30530][SQL] Fix filter pushdown for bad CSV records
### What changes were proposed in this pull request?
In the PR, I propose to fix the bug reported in SPARK-30530. CSV datasource returns invalid records in the case when `parsedSchema` is shorter than number of tokens returned by UniVocity parser. In the case `UnivocityParser.convert()` always throws `BadRecordException` independently from the result of applying filters.

For the described case, I propose to save the exception in `badRecordException` and continue value conversion according to `parsedSchema`. If a bad record doesn't pass filters, `convert()` returns empty Seq otherwise throws `badRecordException`.

### Why are the changes needed?
It fixes the bug reported in the JIRA ticket.

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

### How was this patch tested?
Added new test from the JIRA ticket.

Closes #27239 from MaxGekk/spark-30530-csv-filter-is-null.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 17:22:38 +08:00
Kent Yao 17857f9b8b [SPARK-30551][SQL] Disable comparison for interval type
### What changes were proposed in this pull request?

As we are not going to follow ANSI to implement year-month and day-time interval types, it is weird to compare the year-month part to the day-time part for our current implementation of interval type now.

Additionally, the current ordering logic comes from PostgreSQL where the implementation of the interval is messy. And we are not aiming PostgreSQL compliance at all.

THIS PR will revert https://github.com/apache/spark/pull/26681 and https://github.com/apache/spark/pull/26337

### Why are the changes needed?

make interval type more future-proofing

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

there are new in 3.0, so no

### How was this patch tested?

existing uts shall work

Closes #27262 from yaooqinn/SPARK-30551.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 15:27:51 +08:00
jiake 0d99d7e3f2 [SPARK-30524] [SQL] follow up SPARK-30524 to resolve comments
### What changes were proposed in this pull request?
Resolve the remaining comments in [PR#27226](https://github.com/apache/spark/pull/27226).

### Why are the changes needed?
Resolve the comments.

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

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

Closes #27253 from JkSelf/followup-skewjoinoptimization2.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 15:10:05 +08:00
Sean Owen 789a4abfa9 [MINOR][HIVE] Pick up HIVE-22708 HTTP transport fix
### What changes were proposed in this pull request?

Pick up the HTTP fix from https://issues.apache.org/jira/browse/HIVE-22708

### Why are the changes needed?

This is a small but important fix to digest handling we should pick up from Hive.

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

No.

### How was this patch tested?

Existing tests

Closes #27273 from srowen/Hive22708.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-18 11:50:59 -08:00
HyukjinKwon a6bdea3ad4 [SPARK-30539][PYTHON][SQL] Add DataFrame.tail in PySpark
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/26809 added `Dataset.tail` API. It should be good to have it in PySpark API as well.

### Why are the changes needed?

To support consistent APIs.

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

No. It adds a new API.

### How was this patch tested?

Manually tested and doctest was added.

Closes #27251 from HyukjinKwon/SPARK-30539.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-18 00:18:12 -08:00
Kevin Yu 96a344511e [SPARK-25993][SQL][TESTS] Add test cases for CREATE EXTERNAL TABLE with subdirectories
### What changes were proposed in this pull request?

This PR aims to add these test cases for resolution of ORC table location reported by [SPARK-25993](https://issues.apache.org/jira/browse/SPARK-25993)
also add corresponding test cases for Parquet table.

### Why are the changes needed?

The current behavior is complex, this test case suites are designed to prevent the accidental behavior change. This pr is rebased on master, the original pr is [23108](https://github.com/apache/spark/pull/23108)

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

No. This adds test cases only.

### How was this patch tested?

This is a new test case.

Closes #27130 from kevinyu98/spark-25993-2.

Authored-by: Kevin Yu <qyu@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-17 17:17:29 -08:00
Gabor Somogyi abf759a91e [SPARK-29876][SS] Delete/archive file source completed files in separate thread
### What changes were proposed in this pull request?
[SPARK-20568](https://issues.apache.org/jira/browse/SPARK-20568) added the possibility to clean up completed files in streaming query. Deleting/archiving uses the main thread which can slow down processing. In this PR I've created thread pool to handle file delete/archival. The number of threads can be configured with `spark.sql.streaming.fileSource.cleaner.numThreads`.

### Why are the changes needed?
Do file delete/archival in separate thread.

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

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

Closes #26502 from gaborgsomogyi/SPARK-29876.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-17 10:45:36 -08:00
Luca Canali fd308ade52 [SPARK-30041][SQL][WEBUI] Add Codegen Stage Id to Stage DAG visualization in Web UI
### What changes were proposed in this pull request?
SPARK-29894 provides information on the Codegen Stage Id in WEBUI for SQL Plan graphs. Similarly, this proposes to add Codegen Stage Id in the DAG visualization for Stage execution. DAGs for Stage execution are available in the WEBUI under the Jobs and Stages tabs.

### Why are the changes needed?
This is proposed as an aid for drill-down analysis of complex SQL statement execution, as it is not always easy to match parts of the SQL Plan graph with the corresponding Stage DAG execution graph. Adding Codegen Stage Id for WholeStageCodegen operations makes this task easier.

### Does this PR introduce any user-facing change?
Stage DAG visualization in the WEBUI will show codegen stage id for WholeStageCodegen operations, as in the example snippet from the WEBUI, Jobs tab  (the query used in the example is TPCDS 2.4 q14a):
![](https://issues.apache.org/jira/secure/attachment/12987461/Snippet_StagesDags_with_CodegenId%20_annotated.png)

### How was this patch tested?
Manually tested, see also example snippet.

Closes #26675 from LucaCanali/addCodegenStageIdtoStageGraph.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-18 01:00:45 +08:00
Terry Kim 64fe192fef [SPARK-30282][SQL] Migrate SHOW TBLPROPERTIES to new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the SHOW TBLPROPERTIES command. This PR along with #27243 should update all the existing V2 commands with `UnresolvedV2Relation`.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-2990](https://issues.apache.org/jira/browse/SPARK-29900).

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

Yes `SHOW TBLPROPERTIES temp_view` now fails with `AnalysisException` will be thrown with a message `temp_view is a temp view not table`. Previously, it was returning empty row.

### How was this patch tested?

Existing tests

Closes #26921 from imback82/consistnet_v2command.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-17 16:51:44 +08:00
Wenchen Fan 0bd7a3dfab [SPARK-29572][SQL] add v1 read fallback API in DS v2
### What changes were proposed in this pull request?

Add a `V1Scan` interface, so that data source v1 implementations can migrate to DS v2 much easier.

### Why are the changes needed?

It's a lot of work to migrate v1 sources to DS v2. The new API added here can allow v1 sources to go through v2 code paths without implementing all the Batch, Stream, PartitionReaderFactory, ... stuff.

We already have a v1 write fallback API after https://github.com/apache/spark/pull/25348

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

no

### How was this patch tested?

new test suite

Closes #26231 from cloud-fan/v1-read-fallback.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-17 12:40:51 +08:00
Maxim Gekk 1a9de8c31f [SPARK-30499][SQL] Remove SQL config spark.sql.execution.pandas.respectSessionTimeZone
### What changes were proposed in this pull request?
In the PR, I propose to remove the SQL config `spark.sql.execution.pandas.respectSessionTimeZone` which has been deprecated since Spark 2.3.

### Why are the changes needed?
To improve code maintainability.

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

### How was this patch tested?
by running python tests, https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven-or-sbt

Closes #27218 from MaxGekk/remove-respectSessionTimeZone.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-17 11:44:49 +09:00
jiake 6e5b4bf113 [SPARK-30524][SQL] Disable OptimizeSkewedJoin rule when introducing additional shuffle
### What changes were proposed in this pull request?
`OptimizeSkewedJoin `rule change the `outputPartitioning `after inserting `PartialShuffleReaderExec `or `SkewedPartitionReaderExec`. So it may  need to introduce additional to ensure the right result. This PR disable `OptimizeSkewedJoin  ` rule when introducing additional shuffle.

### Why are the changes needed?
bug fix

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

### How was this patch tested?
Add new ut

Closes #27226 from JkSelf/followup-skewedoptimization.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 22:52:00 +08:00
Kent Yao 82f25f5855 [SPARK-30507][SQL] TableCalalog reserved properties shoudn't be changed via options or tblpropeties
### What changes were proposed in this pull request?

TableCatalog reserves some properties, e,g `provider`, `location` for internal usage. Some of them are static once create, some of them need specific syntax to modify. Instead of using `OPTIONS (k='v')` or TBLPROPERTIES (k='v'), if k is a reserved TableCatalog property, we should use its specific syntax to add/modify/delete it. e.g. `provider` is a reserved property, we should use the `USING` clause to specify it, and should not allow `ALTER TABLE ... UNSET TBLPROPERTIES('provider')` to delete it. Also, there are two paths for v1/v2 catalog tables to resolve these properties, e.g. the v1 session catalog tables will only use the `USING` clause to decide `provider` but v2 tables will also lookup OPTION/TBLPROPERTIES(although there is a bug prohibit it).

Additionally, 'path' is not reserved but holds special meaning for `LOCATION` and it is used in `CREATE/REPLACE TABLE`'s `OPTIONS` sub-clause. Now for the session catalog tables, the `path` is case-insensitive, but for the non-session catalog tables, it is case-sensitive, we should make it both case insensitive for disambiguation.

### Why are the changes needed?
prevent reserved properties from being modified unexpectedly
unify the property resolution for v1/v2.
fix some bugs.

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

yes
1 . `location` and `provider` (case sensitive) cannot be used in  `CREATE/REPLACE TABLE ... OPTIONS/TBLPROPETIES` and `ALTER TABLE ... SET TBLPROPERTIES (...)`, if legacy on, they will be ignored to let the command success without having side effects
3. Once `path` in `CREATE/REPLACE TABLE ... OPTIONS`  is case insensitive for v1 but sensitive for v2, but now we change it case insensitive for both kinds of tables, then v2 tables will also fail if `LOCATION` and `OPTIONS('PaTh' ='abc')` are both specified or will pick `PaTh`'s value as table location if `LOCATION` is missing.
4. Now we will detect if there are two different case `path` keys or more in  `CREATE/REPLACE TABLE ... OPTIONS`, once it is a kind of unexpected last-win policy for v1, and v2 is case sensitive.

### How was this patch tested?

add ut

Closes #27197 from yaooqinn/SPARK-30507.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-16 21:46:07 +08:00
Maxim Gekk 018bdcc53c [SPARK-30521][SQL][TESTS] Eliminate deprecation warnings for ExpressionInfo
### What changes were proposed in this pull request?
In the PR, I propose to use non-deprecated constructor of `ExpressionInfo` in `SparkSessionExtensionSuite`, and pass valid strings as `examples`, `note`, `since` and `deprecated` parameters.

### Why are the changes needed?
Using another constructor allows to eliminate the following deprecation warnings while compiling Spark:
```
Warning:(335, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction", "usage", "extended usage"),
Warning:(732, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"),
Warning:(751, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information.
    new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"),
```

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

### How was this patch tested?
By compiling and running `SparkSessionExtensionSuite`.

Closes #27221 from MaxGekk/eliminate-expr-info-warnings.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-16 13:36:28 +09:00
Maxim Gekk 4e50f0291f [SPARK-30323][SQL] Support filters pushdown in CSV datasource
### What changes were proposed in this pull request?

In the PR, I propose to support pushed down filters in CSV datasource. The reason of pushing a filter up to `UnivocityParser` is to apply the filter as soon as all its attributes become available i.e. converted from CSV fields to desired values according to the schema. This allows to skip conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of CSV string fields to desired values are comparably expensive ( for example, conversion to `TIMESTAMP` values).

Here are details of the implementation:
- `UnivocityParser.convert()` converts parsed CSV tokens one-by-one sequentially starting from index 0 up to `parsedSchema.length - 1`. At current index `i`, it applies filters that refer to attributes at row fields indexes `0..i`. If any filter returns `false`, it skips conversions of other input tokens.
- Pushed filters are converted to expressions. The expressions are bound to row positions according to `requiredSchema`. The expressions are compiled to predicates via generating Java code.
- To be able to apply predicates to partially initialized rows, the predicates are grouped, and combined via the `And` expression. Final predicate at index `N` can refer to row fields at the positions `0..N`, and can be applied to a row even if other fields at the positions `N+1..requiredSchema.lenght-1` are not set.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks more **than 9 times** (on JDK 8 & 11):
```
OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2
Intel(R) Core(TM) i7-4850HQ CPU  2.30GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                       11889          11945          52          0.0      118893.1       1.0X
pushdown disabled                                 11790          11860         115          0.0      117902.3       1.0X
w/ filters                                         1240           1278          33          0.1       12400.8       9.6X
```

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

### How was this patch tested?
- Added new test suite `CSVFiltersSuite`
- Added tests to `CSVSuite` and `UnivocityParserSuite`

Closes #26973 from MaxGekk/csv-filters-pushdown.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-16 13:10:08 +09:00
Liang-Chi Hsieh be4d825872 [SPARK-30312][SQL][FOLLOWUP] Rename conf by adding .enabled
### What changes were proposed in this pull request?

Based on the [comment](https://github.com/apache/spark/pull/26956#discussion_r366680558), this patch changes the SQL config name from `spark.sql.truncateTable.ignorePermissionAcl` to `spark.sql.truncateTable.ignorePermissionAcl.enabled`.

### Why are the changes needed?

Make this config consistent other SQL configs.

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

No.

### How was this patch tested?

Unit test.

Closes #27210 from viirya/truncate-table-permission-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-15 20:09:12 -08:00
yi.wu 5a55a5a0d0 [SPARK-30518][SQL] Precision and scale should be same for values between -1.0 and 1.0 in Decimal
### What changes were proposed in this pull request?

For decimal values between -1.0 and 1.0, it should has same precision and scale in `Decimal`, in order to make it be consistent with `DecimalType`.

### Why are the changes needed?

Currently, for values between -1.0 and 1.0, precision and scale is inconsistent between `Decimal` and `DecimalType`. For example, for numbers like 0.3, it will have (precision, scale) as (2, 1) in `Decimal`, but (1, 1) in `DecimalType`:

```
scala> Literal(new BigDecimal("0.3")).dataType.asInstanceOf[DecimalType].precision
res3: Int = 1

scala> Literal(new BigDecimal("0.3")).value.asInstanceOf[Decimal].precision
res4: Int = 2
```

We should make `Decimal` be consistent with `DecimalType`. And, here, we change it to only count precision digits after dot for values between -1.0 and 1.0 as other DBMS does, like hive:

```
hive> create table testrel as select 0.3;
hive> describe testrel;
OK
_c0                 	decimal(1,1)
```

This could bring larger scale for values between -1.0 and 1.0.

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

No.

### How was this patch tested?

Updated existed tests.

Closes #27217 from Ngone51/set-decimal-from-javadecimal.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-16 11:14:43 +09:00
Takeshi Yamamuro a3a42b30d0 [SPARK-27986][SQL][FOLLOWUP] Respect filter in sql/toString of AggregateExpression
### What changes were proposed in this pull request?

This pr intends to add filter information in the explain output of an aggregate (This is a follow-up of #26656).

Without this pr:
```
scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true)
== Parsed Logical Plan ==
'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)]
+- 'UnresolvedRelation [t]

== Analyzed Logical Plan ==
k: int, sum(v): bigint
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L]
+- SubqueryAlias `default`.`t`
   +- Relation[k#0,v#1] parquet

== Optimized Logical Plan ==
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L]
+- Relation[k#0,v#1] parquet

== Physical Plan ==
HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v)#3L])
+- Exchange hashpartitioning(k#0, 200), true, [id=#20]
   +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint))], output=[k#0, sum#7L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int,v:int>

scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show()
+---+------+
|  k|sum(v)|
+---+------+
+---+------+
```

With this pr:
```
scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true)
== Parsed Logical Plan ==
'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)]
+- 'UnresolvedRelation [t]

== Analyzed Logical Plan ==
k: int, sum(v) FILTER (v > 3): bigint
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L]
+- SubqueryAlias `default`.`t`
   +- Relation[k#0,v#1] parquet

== Optimized Logical Plan ==
Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L]
+- Relation[k#0,v#1] parquet

== Physical Plan ==
HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v) FILTER (v > 3)#5L])
+- Exchange hashpartitioning(k#0, 200), true, [id=#20]
   +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint)) filter (v#1 > 3)], output=[k#0, sum#9L])
      +- *(1) ColumnarToRow
         +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<k:int,v:int>

scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show()
+---+---------------------+
|  k|sum(v) FILTER (v > 3)|
+---+---------------------+
+---+---------------------+
```

### Why are the changes needed?

For better usability.

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

No.

### How was this patch tested?

Manually.

Closes #27198 from maropu/SPARK-27986-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-16 11:11:36 +09:00
Wenchen Fan 883ae331c3 [SPARK-30497][SQL] migrate DESCRIBE TABLE to the new framework
### What changes were proposed in this pull request?

Use the new framework to resolve the DESCRIBE TABLE command.

The v1 DESCRIBE TABLE command supports both table and view. Checked with Hive and Presto, they don't have DESCRIBE TABLE syntax but only DESCRIBE, which supports both table and view:
1. https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-DescribeTable/View/MaterializedView/Column
2. https://prestodb.io/docs/current/sql/describe.html

We should make it clear that DESCRIBE support both table and view, by renaming the command to `DescribeRelation`.

This PR also tunes the framework a little bit to support the case that a command accepts both table and view.

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: SPARK-29900.

Note that I make a separate PR here instead of #26921, as I need to update the framework to support a new use case: accept both table and view.

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

no

### How was this patch tested?

existing tests

Closes #27187 from cloud-fan/describe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-15 17:38:52 -08:00
Takeshi Yamamuro 8a926e448f [SPARK-26736][SQL] Partition pruning through nondeterministic expressions in Hive tables
### What changes were proposed in this pull request?

This PR intends to improve partition pruning for nondeterministic expressions in Hive tables:

Before this PR:
```
scala> sql("""create table test(id int) partitioned by (dt string)""")
scala> sql("""select * from test where dt='20190101' and rand() < 0.5""").explain()

== Physical Plan ==
*(1) Filter ((isnotnull(dt#19) AND (dt#19 = 20190101)) AND (rand(6515336563966543616) < 0.5))
+- Scan hive default.test [id#18, dt#19], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#18], [dt#19], Statistics(sizeInBytes=8.0 EiB)
```
After this PR:
```
== Physical Plan ==
*(1) Filter (rand(-9163956883277176328) < 0.5)
+- Scan hive default.test [id#0, dt#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#0], [dt#1], Statistics(sizeInBytes=8.0 EiB), [isnotnull(dt#1), (dt#1 = 20190101)]
```
This PR is the rework of #24118.

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

Unit tests added.

Closes #27219 from maropu/SPARK-26736.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-16 08:36:22 +09:00
Jungtaek Lim (HeartSaVioR) e751bc66a0 [SPARK-30479][SQL] Apply compaction of event log to SQL events
### What changes were proposed in this pull request?

This patch addresses adding event filter to handle SQL related events. This patch is next task of SPARK-29779 (#27085), please refer the description of PR #27085 to see overall rationalization of this patch.

Below functionalities will be addressed in later parts:

* integrate compaction into FsHistoryProvider
* documentation about new configuration

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

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

No.

### How was this patch tested?

Added UTs.

Closes #27164 from HeartSaVioR/SPARK-30479.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 10:47:31 -08:00
Takeshi Yamamuro 5f6cd61913 [SPARK-29708][SQL] Correct aggregated values when grouping sets are duplicated
### What changes were proposed in this pull request?

This pr intends to fix wrong aggregated values in `GROUPING SETS` when there are duplicated grouping sets in a query (e.g., `GROUPING SETS ((k1),(k1))`).

For example;
```
scala> spark.table("t").show()
+---+---+---+
| k1| k2|  v|
+---+---+---+
|  0|  0|  3|
+---+---+---+

scala> sql("""select grouping_id(), k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2))""").show()
+-------------+---+----+------+
|grouping_id()| k1|  k2|sum(v)|
+-------------+---+----+------+
|            0|  0|   0|     9| <---- wrong aggregate value and the correct answer is `3`
|            1|  0|null|     3|
+-------------+---+----+------+

// PostgreSQL case
postgres=#  select k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2));
 k1 |  k2  | sum
----+------+-----
  0 |    0 |   3
  0 |    0 |   3
  0 |    0 |   3
  0 | NULL |   3
(4 rows)

// Hive case
hive> select GROUPING__ID, k1, k2, sum(v) from t group by k1, k2 grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2));
1	0	NULL	3
0	0	0	3
```
[MS SQL Server has the same behaviour with PostgreSQL](https://github.com/apache/spark/pull/26961#issuecomment-573638442). This pr follows the behaviour of PostgreSQL/SQL server; it adds one more virtual attribute in `Expand` for avoiding wrongly grouping rows with the same grouping ID.

### Why are the changes needed?

To fix bugs.

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

No.

### How was this patch tested?

The existing tests.

Closes #26961 from maropu/SPARK-29708.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-15 22:02:16 +09:00
Gengliang Wang 240840fe92 [SPARK-30515][SQL] Refactor SimplifyBinaryComparison to reduce the time complexity
### What changes were proposed in this pull request?

The changes in the rule `SimplifyBinaryComparison` from https://github.com/apache/spark/pull/27008 could bring performance regression in the optimizer when there are a large set of filter conditions.

We need to improve the implementation and reduce the time complexity.

### Why are the changes needed?

Need to fix the potential performance regression in the optimizer.

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

No

### How was this patch tested?

Existing unit tests.
Also run a micor benchmark in `BinaryComparisonSimplificationSuite`
```
object Optimize extends RuleExecutor[LogicalPlan] {
    val batches =
      Batch("Constant Folding", FixedPoint(50),
        SimplifyBinaryComparison) :: Nil
  }

test("benchmark") {
  val a = Symbol("a")
  val condition = (1 to 500).map(i => EqualTo(a, a)).reduceLeft(And)
  val finalCondition = And(condition, IsNotNull(a))
  val plan = nullableRelation.where(finalCondition).analyze
  val start = System.nanoTime()
  Optimize.execute(plan)
  println((System.nanoTime() - start) /1000000)
}
```

Before the changes: 2507ms
After the changes: 3ms

Closes #27212 from gengliangwang/SimplifyBinaryComparison.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-15 21:52:43 +09:00
Ajith 0c6bd3bd0b [SPARK-27142][SQL] Provide REST API for SQL information
### What changes were proposed in this pull request?

Currently for Monitoring Spark application SQL information is not available from REST but only via UI. REST provides only applications,jobs,stages,environment. This Jira is targeted to provide a REST API so that SQL level information can be found
A single SQL query can result into multiple jobs. So for end user who is using STS or spark-sql, the intended highest level of probe is the SQL which he has executed. This information can be seen from SQL tab. Attaching a sample.
![image](https://user-images.githubusercontent.com/22072336/54298729-5524a800-45df-11e9-8e4d-b99a8b882031.png)
But same information he cannot access using the REST API exposed by spark and he always have to rely on jobs API which may be difficult. So i intend to expose the information seen in SQL tab in UI via REST API

Mainly:

Id :  Long - execution id of the sql
status : String - possible values COMPLETED/RUNNING/FAILED
description : String - executed SQL string
planDescription : String - Plan representation
metrics : Seq[Metrics] - `Metrics` contain `metricName: String, metricValue: String`
submissionTime : String - formatted `Date` time of SQL submission
duration : Long - total run time in milliseconds
runningJobIds : Seq[Int] - sequence of running job ids
failedJobIds : Seq[Int] - sequence of failed job ids
successJobIds : Seq[Int] - sequence of success job ids

* To fetch sql executions: /sql?details=boolean&offset=integer&length=integer
* To fetch single execution:  /sql/{executionID}?details=boolean

| parameter | type | remarks |
| ------------- |:-------------:| -----|
| details | boolean | Optional. Set true to get plan description and metrics information, defaults to false |
| offset | integer | Optional. offset to fetch the executions, defaults to 0 |
| length | integer | Optional. total number of executions to be fetched, defaults to 20 |

### Why are the changes needed?
To support users query SQL information via REST API

### Does this PR introduce any user-facing change?
Yes. It provides a new monitoring URL for SQL

### How was this patch tested?
Tested manually

![image](https://user-images.githubusercontent.com/22072336/54282168-6d85ca00-45c1-11e9-8935-7586ccf0efff.png)

![image](https://user-images.githubusercontent.com/22072336/54282191-7b3b4f80-45c1-11e9-941c-f0ec37026192.png)

Closes #24076 from ajithme/restapi.

Lead-authored-by: Ajith <ajith2489@gmail.com>
Co-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-14 10:05:47 -08:00
Erik Erlandson 176b69642e [SPARK-30423][SQL] Deprecate UserDefinedAggregateFunction
### What changes were proposed in this pull request?
* Annotate UserDefinedAggregateFunction as deprecated by SPARK-27296
* Update user doc examples to reflect new ability to register typed Aggregator[IN, BUF, OUT] as an untyped aggregating UDF
### Why are the changes needed?
UserDefinedAggregateFunction is being deprecated

### Does this PR introduce any user-facing change?
Changes are to user documentation, and deprecation annotations.

### How was this patch tested?
Testing was via package build to verify doc generation, deprecation warnings, and successful example compilation.

Closes #27193 from erikerlandson/spark-30423.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 22:07:13 +08:00
jiake a2aa966ef6 [SPARK-29544][SQL] optimize skewed partition based on data size
### What changes were proposed in this pull request?
Skew Join is common and can severely downgrade performance of queries, especially those with joins. This PR aim to optimization the skew join based on the runtime Map output statistics by adding "OptimizeSkewedPartitions" rule. And The details design doc is [here](https://docs.google.com/document/d/1NkXN-ck8jUOS0COz3f8LUW5xzF8j9HFjoZXWGGX2HAg/edit). Currently we can support "Inner, Cross, LeftSemi, LeftAnti, LeftOuter, RightOuter" join type.

### Why are the changes needed?
To optimize the skewed partition in runtime based on AQE

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

### How was this patch tested?
UT

Closes #26434 from JkSelf/skewedPartitionBasedSize.

Lead-authored-by: jiake <ke.a.jia@intel.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: JiaKe <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 20:31:44 +08:00
root1 e0efd213eb [SPARK-30292][SQL] Throw Exception when invalid string is cast to numeric type in ANSI mode
### What changes were proposed in this pull request?
If spark.sql.ansi.enabled is set,
throw exception when cast to any numeric type do not follow the ANSI SQL standards.

### Why are the changes needed?
ANSI SQL standards do not allow invalid strings to get casted into numeric types and throw exception for that. Currently spark sql gives NULL in such cases.

Before:
`select cast('str' as decimal)  => NULL`

After :
`select cast('str' as decimal) => invalid input syntax for type numeric: str`

These results are after setting `spark.sql.ansi.enabled=true`

### Does this PR introduce any user-facing change?
Yes. Now when ansi mode is on users will get arithmetic exception for invalid strings.

### How was this patch tested?
Unit Tests Added.

Closes #26933 from iRakson/castDecimalANSI.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 17:03:10 +08:00
Maxim Gekk 88fc8dbc09 [SPARK-30482][SQL][CORE][TESTS] Add sub-class of AppenderSkeleton reusable in tests
### What changes were proposed in this pull request?
In the PR, I propose to define a sub-class of `AppenderSkeleton` in `SparkFunSuite` and reuse it from other tests. The class stores incoming `LoggingEvent` in an array which is available to tests for future analysis of logged events.

### Why are the changes needed?
This eliminates code duplication in tests.

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

### How was this patch tested?
By existing test suites - `CSVSuite`, `OptimizerLoggingSuite`, `JoinHintSuite`, `CodeGenerationSuite` and `SQLConfSuite`.

Closes #27166 from MaxGekk/dedup-appender-skeleton.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-14 16:03:10 +09:00
Maxim Gekk 1846b0261b [SPARK-30500][SPARK-30501][SQL] Remove SQL configs deprecated in Spark 2.1 and 2.3
### What changes were proposed in this pull request?
In the PR, I propose to remove already deprecated SQL configs:
- `spark.sql.variable.substitute.depth` deprecated in Spark 2.1
- `spark.sql.parquet.int64AsTimestampMillis` deprecated in Spark 2.3

Also I moved `removedSQLConfigs` closer to `deprecatedSQLConfigs`. This will allow to have references to other config entries.

### Why are the changes needed?
To improve code maintainability.

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

### How was this patch tested?
By existing test suites `ParquetQuerySuite` and `SQLConfSuite`.

Closes #27169 from MaxGekk/remove-deprecated-conf-2.4.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-14 11:06:48 +09:00
HyukjinKwon 6646b3e13e Revert "[SPARK-28670][SQL] create function should thrown Exception if the resource is not found"
This reverts commit 16e5e79877.
2020-01-14 10:40:35 +09:00
jiake b389b8c5f0 [SPARK-30188][SQL] Resolve the failed unit tests when enable AQE
### What changes were proposed in this pull request?
Fix all the failed tests when enable AQE.

### Why are the changes needed?
Run more tests with AQE to catch bugs, and make it easier to enable AQE by default in the future.

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

### How was this patch tested?
Existing unit tests

Closes #26813 from JkSelf/enableAQEDefault.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-13 22:55:19 +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
ulysses 8ce7962931 [SPARK-30245][SQL] Add cache for Like and RLike when pattern is not static
### What changes were proposed in this pull request?

Add cache for Like and RLike when pattern is not static

### Why are the changes needed?

When pattern is not static, we should avoid compile pattern every time if some pattern is same.
Here is perf numbers, include 3 test groups and use `range` to make it easy.
```
// ---------------------
// 10,000 rows and 10 partitions
val df1 = spark.range(0, 10000, 1, 10).withColumnRenamed("id", "id1")
val df2 = spark.range(0, 10000, 1, 10).withColumnRenamed("id", "id2")

val start = System.currentTimeMillis
df1.join(df2).where("id2 like id1").count()
// before  16939
// after    6352
println(System.currentTimeMillis - start)

// ---------------------
// 10,000 rows and 100 partitions
val df1 = spark.range(0, 10000, 1, 100).withColumnRenamed("id", "id1")
val df2 = spark.range(0, 10000, 1, 100).withColumnRenamed("id", "id2")

val start = System.currentTimeMillis
df1.join(df2).where("id2 like id1").count()
// before  11070
// after    4680
println(System.currentTimeMillis - start)

// ---------------------
// 20,000 rows and 10 partitions
val df1 = spark.range(0, 20000, 1, 10).withColumnRenamed("id", "id1")
val df2 = spark.range(0, 20000, 1, 10).withColumnRenamed("id", "id2")

val start = System.currentTimeMillis
df1.join(df2).where("id2 like id1").count()
// before 66962
// after  29934
println(System.currentTimeMillis - start)
```

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

No.

### How was this patch tested?

Closes #26875 from ulysses-you/SPARK-30245.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-13 15:12:19 +09:00
Dongjoon Hyun 361583d1f5 [SPARK-30409][TEST][FOLLOWUP][HOTFIX] Remove dangling JSONBenchmark-jdk11-results.txt
### What changes were proposed in this pull request?

This PR removes a dangling test result, `JSONBenchmark-jdk11-results.txt`.
This causes a case-sensitive issue on Mac.

```
$ git clone https://gitbox.apache.org/repos/asf/spark.git spark-gitbox
Cloning into 'spark-gitbox'...
remote: Counting objects: 671717, done.
remote: Compressing objects: 100% (258021/258021), done.
remote: Total 671717 (delta 329181), reused 560390 (delta 228097)
Receiving objects: 100% (671717/671717), 149.69 MiB | 950.00 KiB/s, done.
Resolving deltas: 100% (329181/329181), done.
Updating files: 100% (16090/16090), done.
warning: the following paths have collided (e.g. case-sensitive paths
on a case-insensitive filesystem) and only one from the same
colliding group is in the working tree:

  'sql/core/benchmarks/JSONBenchmark-jdk11-results.txt'
  'sql/core/benchmarks/JsonBenchmark-jdk11-results.txt'
```

### Why are the changes needed?

Previously, since the file name didn't match with `object JSONBenchmark`, it made a confusion when we ran the benchmark. So, 4e0e4e51c4 renamed `JSONBenchmark` to `JsonBenchmark`. However, at the same time frame, https://github.com/apache/spark/pull/26003 regenerated this file.

Recently, https://github.com/apache/spark/pull/27078 regenerates the results with the correct file name, `JsonBenchmark-jdk11-results.txt`. So, we can remove the old one.

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

No. This is a test result.

### How was this patch tested?

Manually check the following correctly generated files in the master. And, check this PR removes the dangling one.
- https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-results.txt
- https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt

Closes #27180 from dongjoon-hyun/SPARK-REMOVE.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-12 23:45:31 +00: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
Erik Erlandson 1f50a5875b [SPARK-27296][SQL] Allows Aggregator to be registered as a UDF
## What changes were proposed in this pull request?
Defines a new subclass of UDF: `UserDefinedAggregator`. Also allows `Aggregator` to be registered as a udf.  Under the hood, the implementation is based on the internal `TypedImperativeAggregate` class that spark's predefined aggregators make use of. The effect is that custom user defined aggregators are now serialized only on partition boundaries instead of being serialized and deserialized at each input row.

The two new modes of using `Aggregator` are as follows:
```scala
val agg: Aggregator[IN, BUF, OUT] = // typed aggregator
val udaf1 = UserDefinedAggregator(agg)
val udaf2 = spark.udf.register("agg", agg)
```

## How was this patch tested?
Unit testing has been added that corresponds to the testing suites for `UserDefinedAggregateFunction`. Additionally, unit tests explicitly count the number of aggregator ser/de cycles to ensure that it is governed only by the number of data partitions.

To evaluate the performance impact, I did two comparisons.
The code and REPL results are recorded on [this gist](https://gist.github.com/erikerlandson/b0e106a4dbaf7f80b4f4f3a21f05f892)
To characterize its behavior I benchmarked both a relatively simple aggregator and then an aggregator with a complex structure (a t-digest).

### performance
The following compares the new `Aggregator` based aggregation against UDAF. In this scenario, the new aggregation is about 100x faster. The difference in performance impact depends on the complexity of the aggregator. For very simple aggregators (e.g. implementing 'sum', etc), the performance impact is more like 25-30%.

```scala
scala> import scala.util.Random._, org.apache.spark.sql.Row, org.apache.spark.tdigest._
import scala.util.Random._
import org.apache.spark.sql.Row
import org.apache.spark.tdigest._

scala> val data = sc.parallelize(Vector.fill(50000){(nextInt(2), nextGaussian, nextGaussian.toFloat)}, 5).toDF("cat", "x1", "x2")
data: org.apache.spark.sql.DataFrame = [cat: int, x1: double ... 1 more field]

scala> val udaf = TDigestUDAF(0.5, 0)
udaf: org.apache.spark.tdigest.TDigestUDAF = TDigestUDAF(0.5,0)

scala> val bs = Benchmark.sample(10) { data.agg(udaf($"x1"), udaf($"x2")).first }
bs: Array[(Double, org.apache.spark.sql.Row)] = Array((16.523,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.3687811...

scala> bs.map(_._1)
res0: Array[Double] = Array(16.523, 17.138, 17.863, 17.801, 17.769, 17.786, 17.744, 17.8, 17.939, 17.854)

scala> val agg = TDigestAggregator(0.5, 0)
agg: org.apache.spark.tdigest.TDigestAggregator = TDigestAggregator(0.5,0)

scala> val udaa = spark.udf.register("tdigest", agg)
udaa: org.apache.spark.sql.expressions.UserDefinedAggregator[Double,org.apache.spark.tdigest.TDigestSQL,org.apache.spark.tdigest.TDigestSQL] = UserDefinedAggregator(TDigestAggregator(0.5,0),None,true,true)

scala> val bs = Benchmark.sample(10) { data.agg(udaa($"x1"), udaa($"x2")).first }
bs: Array[(Double, org.apache.spark.sql.Row)] = Array((0.313,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.36878118...

scala> bs.map(_._1)
res1: Array[Double] = Array(0.313, 0.193, 0.175, 0.185, 0.174, 0.176, 0.16, 0.186, 0.171, 0.179)

scala>
```

Closes #25024 from erikerlandson/spark-27296.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-12 15:18:30 +08:00
ulysses 823e3d309c [SPARK-30353][SQL] Add IsNotNull check in SimplifyBinaryComparison optimization
### What changes were proposed in this pull request?

Now Spark can propagate constraint during sql optimization when `spark.sql.constraintPropagation.enabled` is true, then `where c = 1` will convert to `where c = 1 and c is not null`. We also can use constraint in `SimplifyBinaryComparison`.

`SimplifyBinaryComparison` will simplify expression which is not nullable and semanticEquals. And we also can simplify if one expression is infered `IsNotNull`.

### Why are the changes needed?

Simplify SQL.
```
create table test (c1 string);

explain extended select c1 from test where c1 = c1 limit 10;
-- before
GlobalLimit 10
+- LocalLimit 10
   +- Filter (isnotnull(c1#20) AND (c1#20 = c1#20))
      +- Relation[c1#20]
-- after
GlobalLimit 10
+- LocalLimit 10
    +- Filter (isnotnull(c1#20)
        +- Relation[c1#20]

explain extended select c1 from test where c1 > c1 limit 10;
-- before
GlobalLimit 10
+- LocalLimit 10
   +- Filter (isnotnull(c1#20) && (c1#20 > c1#20))
      +- Relation[c1#20]
-- after
LocalRelation <empty>, [c1#20]
```

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

No.

### How was this patch tested?

Add UT.

Closes #27008 from ulysses-you/SPARK-30353.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-12 15:03:57 +08:00
Liang-Chi Hsieh b04407169b [SPARK-30312][SQL][FOLLOWUP] Use inequality check instead to be robust
### What changes were proposed in this pull request?

This is a followup to fix a brittle assert in a test case.

### Why are the changes needed?

Original assert assumes that default permission is `rwxr-xr-x`, but in jenkins [env](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-1.2/6/testReport/junit/org.apache.spark.sql.execution.command/InMemoryCatalogedDDLSuite/SPARK_30312__truncate_table___keep_acl_permission/) it could be `rwxrwxr-x`.

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

No

### How was this patch tested?

Unit test.

Closes #27175 from viirya/hot-fix.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-11 13:19:04 -08:00
Liang-Chi Hsieh b5bc3e12a6 [SPARK-30312][SQL] Preserve path permission and acl when truncate table
### What changes were proposed in this pull request?

This patch proposes to preserve existing permission/acls of paths when truncate table/partition.

### Why are the changes needed?

When Spark SQL truncates table, it deletes the paths of table/partitions, then re-create new ones. If permission/acls were set on the paths, the existing permission/acls will be deleted.

We should preserve the permission/acls if possible.

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

Yes. When truncate table/partition, Spark will keep permission/acls of paths.

### How was this patch tested?

Unit test.

Manual test:

1. Create a table.
2. Manually change it permission/acl
3. Truncate table
4. Check permission/acl

```scala
val df = Seq(1, 2, 3).toDF
df.write.mode("overwrite").saveAsTable("test.test_truncate_table")
val testTable = spark.table("test.test_truncate_table")
testTable.show()
+-----+
|value|
+-----+
|    1|
|    2|
|    3|
+-----+
// hdfs dfs -setfacl ...
// hdfs dfs -getfacl ...
sql("truncate table test.test_truncate_table")
// hdfs dfs -getfacl ...
val testTable2 = spark.table("test.test_truncate_table")
testTable2.show()
+-----+
|value|
+-----+
+-----+
```

![Screen Shot 2019-12-30 at 3 12 15 PM](https://user-images.githubusercontent.com/68855/71604577-c7875a00-2b17-11ea-913a-ba88096d20ab.jpg)

Closes #26956 from viirya/truncate-table-permission.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-10 11:46:28 -08:00
Zhenhua Wang 2bd8731813 [SPARK-30468][SQL] Use multiple lines to display data columns for show create table command
### What changes were proposed in this pull request?
Currently data columns are displayed in one line for show create table command, when the table has many columns (to make things even worse, columns may have long names or comments), the displayed result is really hard to read.

To improve readability, we print each column in a separate line. Note that other systems like Hive/MySQL also display in this way.

Also, for data columns, table properties and options, we put the right parenthesis to the end of the last column/property/option, instead of occupying a separate line.

### Why are the changes needed?
for better readability

### Does this PR introduce any user-facing change?
before the change:
```
spark-sql> show create table test_table;
CREATE TABLE `test_table` (`col1` INT COMMENT 'This is comment for column 1', `col2` STRING COMMENT 'This is comment for column 2', `col3` DOUBLE COMMENT 'This is comment for column 3')
USING parquet
OPTIONS (
  `bar` '2',
  `foo` '1'
)
TBLPROPERTIES (
  'a' = 'x',
  'b' = 'y'
)
```
after the change:
```
spark-sql> show create table test_table;
CREATE TABLE `test_table` (
  `col1` INT COMMENT 'This is comment for column 1',
  `col2` STRING COMMENT 'This is comment for column 2',
  `col3` DOUBLE COMMENT 'This is comment for column 3')
USING parquet
OPTIONS (
  `bar` '2',
  `foo` '1')
TBLPROPERTIES (
  'a' = 'x',
  'b' = 'y')
```

### How was this patch tested?
modified existing tests

Closes #27147 from wzhfy/multi_line_columns.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-10 10:55:53 -06:00
Takeshi Yamamuro b942832bd3 [SPARK-30343][SQL] Skip unnecessary checks in RewriteDistinctAggregates
### What changes were proposed in this pull request?

This pr intends to skip the unnecessary checks that most aggregate quries don't need in RewriteDistinctAggregates.

### Why are the changes needed?

For minor optimization.

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

No.

### How was this patch tested?

Existing tests.

Closes #26997 from maropu/OptDistinctAggRewrite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-10 23:33:08 +09:00
root1 2a629e5d10 [SPARK-30234][SQL] ADD FILE cannot add directories from sql CLI
### What changes were proposed in this pull request?
Now users can add directories from sql CLI as well using ADD FILE command and setting spark.sql.addDirectory.recursive to true.

### Why are the changes needed?
In SPARK-4687, support was added for adding directories as resources. But sql users cannot use that feature from CLI.

`ADD FILE /path/to/folder` gives the following error:
`org.apache.spark.SparkException: Added file /path/to/folder is a directory and recursive is not turned on.`

Users need to turn on `recursive` for adding directories. Thus a configuration was required which will allow users to turn on `recursive`.
Also Hive allow users to add directories from their shell.

### Does this PR introduce any user-facing change?
Yes. Users can set recursive using `spark.sql.addDirectory.recursive`.

### How was this patch tested?
Manually.
Will add test cases soon.

 SPARK SCREENSHOTS
When `spark.sql.addDirectory.recursive` is not turned on.
![Screenshot from 2019-12-13 08-02-13](https://user-images.githubusercontent.com/15366835/70765124-c6b4a100-1d7f-11ea-9352-9c010af5b38b.png)

After setting `spark.sql.addDirectory.recursive` to true.

![Screenshot from 2019-12-13 08-02-59](https://user-images.githubusercontent.com/15366835/70765118-be5c6600-1d7f-11ea-9faf-0b1c46ee299b.png)

HIVE SCREENSHOT

![Screenshot from 2019-12-13 14-44-41](https://user-images.githubusercontent.com/15366835/70788979-17e08700-1db8-11ea-9c0c-b6d6f6e80a35.png)

`RELEASE_NOTES.txt` is text file while `dummy` is a directory.

Closes #26863 from iRakson/SPARK-30234.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 22:36:45 +09:00
Peter Toth 418f7dc973 [SPARK-30447][SQL] Constant propagation nullability issue
## What changes were proposed in this pull request?

This PR fixes `ConstantPropagation` rule as the current implementation produce incorrect results in some cases. E.g.
```
SELECT * FROM t WHERE NOT(c = 1 AND c + 1 = 1)
```
returns those rows where `c` is null due to `1 + 1 = 1` propagation but it shouldn't.

## Why are the changes needed?

To fix a bug.

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

Yes, fixes a bug.

## How was this patch tested?

New UTs.

Closes #27119 from peter-toth/SPARK-30447.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-10 21:42:10 +09:00
Kent Yao bcf07cbf5f [SPARK-30018][SQL] Support ALTER DATABASE SET OWNER syntax
### What changes were proposed in this pull request?
In this pull request, we are going to support `SET OWNER` syntax for databases and namespaces,

```sql
ALTER (DATABASE|SCHEME|NAMESPACE) database_name SET OWNER [USER|ROLE|GROUP] user_or_role_group;
```
Before this commit 332e252a14, we didn't care much about ownerships for the catalog objects. In 332e252a14, we determined to use properties to store ownership staff, and temporarily used `alter database ... set dbproperties ...` to support switch ownership of a database. This PR aims to use the formal syntax to replace it.

In hive, `ownerName/Type` are fields of the database objects, also they can be normal properties.
```
create schema test1 with dbproperties('ownerName'='yaooqinn')
```
The create/alter database syntax will not change the owner to `yaooqinn` but store it in parameters. e.g.
```
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
| db_name  | comment  |                           location                            | owner_name  | owner_type  |      parameters       |
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
| test1    |          | hdfs://quickstart.cloudera:8020/user/hive/warehouse/test1.db  | anonymous   | USER        | {ownerName=yaooqinn}  |
+----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+
```
In this pull request, because we let the `ownerName` become reversed, so it will neither change the owner nor store in dbproperties, just be omitted silently.

## Why are the changes needed?

Formal syntax support for changing database ownership

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

yes, add a new syntax

### How was this patch tested?

add unit tests

Closes #26775 from yaooqinn/SPARK-30018.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-10 16:47:08 +08:00
Wenchen Fan 0ec0355611 [SPARK-30439][SQL] Support non-nullable column in CREATE TABLE, ADD COLUMN and ALTER TABLE
### What changes were proposed in this pull request?

Allow users to specify NOT NULL in CREATE TABLE and ADD COLUMN column definition, and add a new SQL syntax to alter column nullability: ALTER TABLE ... ALTER COLUMN SET/DROP NOT NULL. This is a SQL standard syntax:
```
<alter column definition> ::=
  ALTER [ COLUMN ] <column name> <alter column action>

<alter column action> ::=
    <set column default clause>
  | <drop column default clause>
  | <set column not null clause>
  | <drop column not null clause>
  | ...

<set column not null clause> ::=
  SET NOT NULL

<drop column not null clause> ::=
  DROP NOT NULL
```

### Why are the changes needed?

Previously we don't support it because the table schema in hive catalog are always nullable. Since we have catalog plugin now, it makes more sense to support NOT NULL at spark side, and let catalog implementations to decide if they support it or not.

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

Yes, this is a new feature

### How was this patch tested?

new tests

Closes #27110 from cloud-fan/nullable.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 10:34:46 +09:00
Maxim Gekk 1ffa627ffb [SPARK-30416][SQL] Log a warning for deprecated SQL config in set() and unset()
### What changes were proposed in this pull request?
1. Put all deprecated SQL configs the map `SQLConf.deprecatedSQLConfigs` with extra info about when configs were deprecated and additional comments that explain why a config was deprecated, what an user can use instead of it. Here is the list of already deprecated configs:
    - spark.sql.hive.verifyPartitionPath
    - spark.sql.execution.pandas.respectSessionTimeZone
    - spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName
    - spark.sql.parquet.int64AsTimestampMillis
    - spark.sql.variable.substitute.depth
    - spark.sql.execution.arrow.enabled
    - spark.sql.execution.arrow.fallback.enabled

2. Output warning in `set()` and `unset()` about deprecated SQL configs

### Why are the changes needed?
This should improve UX with Spark SQL and notify users about already deprecated SQL configs.

### Does this PR introduce any user-facing change?
Yes, before:
```
spark-sql> set spark.sql.hive.verifyPartitionPath=true;
spark.sql.hive.verifyPartitionPath	true
```
After:
```
spark-sql> set spark.sql.hive.verifyPartitionPath=true;
20/01/03 21:28:17 WARN RuntimeConfig: The SQL config 'spark.sql.hive.verifyPartitionPath' has been deprecated in Spark v3.0.0 and may be removed in the future. This config is replaced by spark.files.ignoreMissingFiles.
spark.sql.hive.verifyPartitionPath	true
```

### How was this patch tested?
Add new test which registers new log appender and catches all logging to check that `set()` and `unset()` log any warning.

Closes #27092 from MaxGekk/group-deprecated-sql-configs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-10 10:32:36 +09:00
shane knapp 4d23938893 [MINOR][SQL][TEST-HIVE1.2] Fix scalastyle error due to length line in hive-1.2 profile
### What changes were proposed in this pull request?

fixing a broken build:
https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-1.2/3/console

### Why are the changes needed?

the build is teh borked!

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

newp

### How was this patch tested?

by the build system

Closes #27156 from shaneknapp/fix-scala-style.

Authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: shane knapp <incomplete@gmail.com>
2020-01-09 15:28:45 -08:00
yi.wu c0e9f9ffb1 [SPARK-30459][SQL] Fix ignoreMissingFiles/ignoreCorruptFiles in data source v2
### What changes were proposed in this pull request?

Fix ignoreMissingFiles/ignoreCorruptFiles in DSv2:

When `FilePartitionReader` finds a missing or corrupt file, it should just skip and continue to read next file rather than stop with current behavior.

### Why are the changes needed?

ignoreMissingFiles/ignoreCorruptFiles in DSv2 is wrong comparing to DSv1.

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

No.

### How was this patch tested?

Updated existed test for `ignoreMissingFiles`. Note I didn't update tests for `ignoreCorruptFiles`, because  there're various datasources has tests for `ignoreCorruptFiles`. So I'm not sure if it's worth to touch all those tests since the basic logic of `ignoreCorruptFiles` should be same with `ignoreMissingFiles`.

Closes #27136 from Ngone51/improve-missing-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-09 11:35:29 -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
Kent Yao c37312342e [SPARK-30183][SQL] Disallow to specify reserved properties in CREATE/ALTER NAMESPACE syntax
### What changes were proposed in this pull request?
Currently, COMMENT and LOCATION are reserved properties for Datasource v2 namespaces. They can be set via specific clauses and via properties. And the ones specified in clauses take precede of properties. Since they are reserved, which means they are not able to visit directly. They should be used in COMMENT/LOCATION clauses ONLY.

### Why are the changes needed?
make reserved properties be reserved.

### Does this PR introduce any user-facing change?
yes, 'location', 'comment' are not allowed use in db properties

### How was this patch tested?
UNIT tests.

Closes #26806 from yaooqinn/SPARK-30183.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-09 10:52:36 +08:00
HyukjinKwon ee8d661058 [SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?

This PR proposes to move pandas related functionalities into pandas package. Namely:

```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py  # Conversion between pandas <> PySpark DataFrames
├── functions.py   # pandas_udf
├── group_ops.py   # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py     # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py       # Type utils between pandas <> PyArrow
└── utils.py       # Version requirement checks
```

In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:

```python
class PandasMapOpsMixin(object):
    def mapInPandas(self, ...):
        ...
        return ...

    # other Pandas <> PySpark APIs
```

```python
class DataFrame(PandasMapOpsMixin):

    # other DataFrame APIs equivalent to Scala side.

```

Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.

### Why are the changes needed?

There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.

Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.

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

No.

### How was this patch tested?

Existing tests should cover. Also, I manually built the PySpark API documentation and checked.

Closes #27109 from HyukjinKwon/pandas-refactoring.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 10:22:50 +09:00
maryannxue af2d3d0179 [SPARK-30315][SQL] Add adaptive execution context
### What changes were proposed in this pull request?
This is a minor code refactoring PR. It creates an adaptive execution context class to wrap objects shared across main query and sub-queries.

### Why are the changes needed?
This refactoring will improve code readability and reduce the number of parameters used to initialize `AdaptiveSparkPlanExec`.

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

### How was this patch tested?
Passed existing UTs.

Closes #26959 from maryannxue/aqe-context.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-08 16:11:46 -08:00
Jungtaek Lim (HeartSaVioR) bd7510bcb7 [SPARK-30281][SS] Consider partitioned/recursive option while verifying archive path on FileStreamSource
### What changes were proposed in this pull request?

This patch renews the verification logic of archive path for FileStreamSource, as we found the logic doesn't take partitioned/recursive options into account.

Before the patch, it only requires the archive path to have depth more than 2 (two subdirectories from root), leveraging the fact FileStreamSource normally reads the files where the parent directory matches the pattern or the file itself matches the pattern. Given 'archive' operation moves the files to the base archive path with retaining the full path, archive path is tend to be safe if the depth is more than 2, meaning FileStreamSource doesn't re-read archived files as new source files.

WIth partitioned/recursive options, the fact is invalid, as FileStreamSource can read any files in any depth of subdirectories for source pattern. To deal with this correctly, we have to renew the verification logic, which may not intuitive and simple but works for all cases.

The new verification logic prevents both cases:

1) archive path matches with source pattern as "prefix" (the depth of archive path > the depth of source pattern)

e.g.
* source pattern: `/hello*/spar?`
* archive path: `/hello/spark/structured/streaming`

Any files in archive path will match with source pattern when recursive option is enabled.

2) source pattern matches with archive path as "prefix" (the depth of source pattern > the depth of archive path)

e.g.
* source pattern: `/hello*/spar?/structured/hello2*`
* archive path: `/hello/spark/structured`

Some archive files will not match with source pattern, e.g. file path:  `/hello/spark/structured/hello2`, then final archived path: `/hello/spark/structured/hello/spark/structured/hello2`.

But some other archive files will still match with source pattern, e.g. file path: `/hello2/spark/structured/hello2`, then final archived path: `/hello/spark/structured/hello2/spark/structured/hello2` which matches with source pattern when recursive is enabled.

Implicitly it also prevents archive path matches with source pattern as full match (same depth).

We would want to prevent any source files to be archived and added to new source files again, so the patch takes most restrictive approach to prevent the possible cases.

### Why are the changes needed?

Without this patch, there's a chance archived files are included as new source files when partitioned/recursive option is enabled, as current condition doesn't take these options into account.

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

Only for Spark 3.0.0-preview (only preview 1 for now, but possibly preview 2 as well) - end users are required to provide archive path with ensuring a bit complicated conditions, instead of simply higher than 2 depths.

### How was this patch tested?

New UT.

Closes #26920 from HeartSaVioR/SPARK-30281.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-08 09:15:41 -08:00
zhengruifeng a93b996635 [MINOR][ML][INT] Array.fill(0) -> Array.ofDim; Array.empty -> Array.emptyIntArray
### What changes were proposed in this pull request?
1, for primitive types `Array.fill(n)(0)` -> `Array.ofDim(n)`;
2, for `AnyRef` types `Array.fill(n)(null)` -> `Array.ofDim(n)`;
3, for primitive types `Array.empty[XXX]` -> `Array.emptyXXXArray`

### Why are the changes needed?
`Array.ofDim` avoid assignments;
`Array.emptyXXXArray` avoid create new object;

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

### How was this patch tested?
existing testsuites

Closes #27133 from zhengruifeng/minor_fill_ofDim.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 00:07:42 +09:00
Zhenhua Wang fa36966b1e [SPARK-30410][SQL] Calculating size of table with large number of partitions causes flooding logs
### What changes were proposed in this pull request?

For a partitioned table, if the number of partitions are very large, e.g. tens of thousands or even larger, calculating its total size causes flooding logs.
The flooding happens in:
1. `calculateLocationSize` prints the starting and ending for calculating the location size, and it is called per partition;
2. `bulkListLeafFiles` prints all partition paths.

This pr is to simplify the logging when calculating the size of a partitioned table.

### How was this patch tested?

not related

Closes #27079 from wzhfy/improve_log.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-08 08:22:38 -06:00
fuwhu 047bff06c3 [SPARK-30215][SQL] Remove PrunedInMemoryFileIndex and merge its functionality into InMemoryFileIndex
### What changes were proposed in this pull request?
Remove PrunedInMemoryFileIndex and merge its functionality into InMemoryFileIndex.

### Why are the changes needed?
PrunedInMemoryFileIndex is only used in CatalogFileIndex.filterPartitions, and its name is kind of confusing, we can completely merge its functionality into InMemoryFileIndex and remove the class.

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

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

Closes #26850 from fuwhu/SPARK-30215.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 20:28:15 +08:00
Terry Kim b2ed6d0b88 [SPARK-30214][SQL][FOLLOWUP] Remove statement logical plans for namespace commands
### What changes were proposed in this pull request?

This is a follow-up to address the following comment: https://github.com/apache/spark/pull/27095#discussion_r363152180

Currently, a SQL command string is parsed to a "statement" logical plan, converted to a logical plan with catalog/namespace, then finally converted to a physical plan. With the new resolution framework, there is no need to create a "statement" logical plan; a logical plan can contain `UnresolvedNamespace` which will be resolved to a `ResolvedNamespace`. This should simply the code base and make it a bit easier to add a new command.

### Why are the changes needed?

Clean up codebase.

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

No

### How was this patch tested?

Existing tests should cover the changes.

Closes #27125 from imback82/SPARK-30214-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 19:33:19 +08:00
Liang-Chi Hsieh 1160457eed [SPARK-30429][SQL] Optimize catalogString and usage in ValidateExternalType.errMsg to avoid OOM
### What changes were proposed in this pull request?

This patch proposes:

1.  Fix OOM at WideSchemaBenchmark: make `ValidateExternalType.errMsg` lazy variable, i.e. not to initiate it in the constructor
2. Truncate `errMsg`: Replacing `catalogString` with `simpleString` which is truncated
3. Optimizing `override def catalogString` in `StructType`: Make `catalogString` more efficient in string generation by using `StringConcat`

### Why are the changes needed?

In the JIRA, it is found that WideSchemaBenchmark fails with OOM, like:
```
[error] Exception in thread "main" org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: validateexternaltype(getexternalrowfield(input[0, org.apac
he.spark.sql.Row, true], 0, a), StructField(b,StructType(StructField(c,StructType(StructField(value_1,LongType,true), StructField(value_10,LongType,true), StructField(value_
100,LongType,true), StructField(value_1000,LongType,true), StructField(value_1001,LongType,true), StructField(value_1002,LongType,true), StructField(value_1003,LongType,true
), StructField(value_1004,LongType,true), StructField(value_1005,LongType,true), StructField(value_1006,LongType,true), StructField(value_1007,LongType,true), StructField(va
lue_1008,LongType,true), StructField(value_1009,LongType,true), StructField(value_101,LongType,true), StructField(value_1010,LongType,true), StructField(value_1011,LongType,
...
ue), StructField(value_99,LongType,true), StructField(value_990,LongType,true), StructField(value_991,LongType,true), StructField(value_992,LongType,true), StructField(value
_993,LongType,true), StructField(value_994,LongType,true), StructField(value_995,LongType,true), StructField(value_996,LongType,true), StructField(value_997,LongType,true),
StructField(value_998,LongType,true), StructField(value_999,LongType,true)),true))
[error]         at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:435)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:408)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:307)
....
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:404)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:307)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUp$1(TreeNode.scala:307)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:307)
[error]         at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.<init>(ExpressionEncoder.scala:198)
[error]         at org.apache.spark.sql.catalyst.encoders.RowEncoder$.apply(RowEncoder.scala:71)
[error]         at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:88)
[error]         at org.apache.spark.sql.SparkSession.internalCreateDataFrame(SparkSession.scala:554)
[error]         at org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:476)
[error]         at org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark$.$anonfun$wideShallowlyNestedStructFieldReadAndWrite$1(WideSchemaBenchmark.scala:126)
...
[error] Caused by: java.lang.OutOfMemoryError: GC overhead limit exceeded
[error]         at java.util.Arrays.copyOf(Arrays.java:3332)
[error]         at java.lang.AbstractStringBuilder.ensureCapacityInternal(AbstractStringBuilder.java:124)
[error]         at java.lang.AbstractStringBuilder.append(AbstractStringBuilder.java:448)
[error]         at java.lang.StringBuilder.append(StringBuilder.java:136)
[error]         at scala.collection.mutable.StringBuilder.append(StringBuilder.scala:213)
[error]         at scala.collection.TraversableOnce.$anonfun$addString$1(TraversableOnce.scala:368)
[error]         at scala.collection.TraversableOnce$$Lambda$67/667447085.apply(Unknown Source)
[error]         at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
[error]         at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
[error]         at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198)
[error]         at scala.collection.TraversableOnce.addString(TraversableOnce.scala:362)
[error]         at scala.collection.TraversableOnce.addString$(TraversableOnce.scala:358)
[error]         at scala.collection.mutable.ArrayOps$ofRef.addString(ArrayOps.scala:198)
[error]         at scala.collection.TraversableOnce.mkString(TraversableOnce.scala:328)
[error]         at scala.collection.TraversableOnce.mkString$(TraversableOnce.scala:327)
[error]         at scala.collection.mutable.ArrayOps$ofRef.mkString(ArrayOps.scala:198)
[error]         at scala.collection.TraversableOnce.mkString(TraversableOnce.scala:330)
[error]         at scala.collection.TraversableOnce.mkString$(TraversableOnce.scala:330)
[error]         at scala.collection.mutable.ArrayOps$ofRef.mkString(ArrayOps.scala:198)
[error]         at org.apache.spark.sql.types.StructType.catalogString(StructType.scala:411)
[error]         at org.apache.spark.sql.catalyst.expressions.objects.ValidateExternalType.<init>(objects.scala:1695)
[error]         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
[error]         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
[error]         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
[error]         at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$makeCopy$7(TreeNode.scala:468)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode$$Lambda$934/387827651.apply(Unknown Source)
[error]         at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$makeCopy$1(TreeNode.scala:467)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode$$Lambda$929/449240381.apply(Unknown Source)
[error]         at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
[error]         at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:435)
```

It is after cb5ea201df commit which refactors `ExpressionEncoder`.

The stacktrace shows it fails at `transformUp` on `objSerializer` in `ExpressionEncoder`. In particular, it fails at initializing `ValidateExternalType.errMsg`, that interpolates `catalogString` of given `expected` data type in a string. In WideSchemaBenchmark we have very deeply nested data type. When we transform on the serializer which contains `ValidateExternalType`, we create redundant big string `errMsg`. Because we just in transforming it and don't use it yet, it is useless and waste a lot of memory.

After make `ValidateExternalType.errMsg` as lazy variable, WideSchemaBenchmark works.

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

No

### How was this patch tested?

Manual test with WideSchemaBenchmark.

Closes #27117 from viirya/SPARK-30429.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-07 18:46:13 -08:00
Zhenhua Wang 9535776e28 [SPARK-30302][SQL] Complete info for show create table for views
### What changes were proposed in this pull request?

Add table/column comments and table properties to the result of show create table of views.

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

When show create table for views, after this patch, the result can contain table/column comments and table properties if they exist.

### How was this patch tested?

add new tests

Closes #26944 from wzhfy/complete_show_create_view.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-08 11:28:37 +09:00
Ajith 2be5286828 [SPARK-30382][SQL] Remove Hive LogUtils usage to prevent ClassNotFoundException
Avoid hive log initialisation as https://github.com/apache/hive/blob/rel/release-2.3.5/common/src/java/org/apache/hadoop/hive/common/LogUtils.java introduces dependency over `org.apache.logging.log4j.core.impl.Log4jContextFactory` which is missing in our spark installer classpath directly. I believe the `LogUtils.initHiveLog4j()` code is here as the HiveServer2 class is copied from Hive.

To make `start-thriftserver.sh --help` command success.

Currently, start-thriftserver.sh --help throws
```
...
Thrift server options:
Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/logging/log4j/spi/LoggerContextFactory
	at org.apache.hive.service.server.HiveServer2.main(HiveServer2.java:167)
	at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:82)
	at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala)
Caused by: java.lang.ClassNotFoundException: org.apache.logging.log4j.spi.LoggerContextFactory
	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)
	... 3 more
```

No

Checked Manually

Closes #27042 from ajithme/thrifthelp.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-07 14:26:04 -08:00
Pablo Langa 9479887ba1 [SPARK-30039][SQL] CREATE FUNCTION should do multi-catalog resolution
### What changes were proposed in this pull request?

Add CreateFunctionStatement and make CREATE FUNCTION go through the same catalog/table resolution framework of v2 commands.

### Why are the changes needed?

It's important to make all the commands have the same table resolution behavior, to avoid confusing
CREATE FUNCTION namespace.function

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

Yes. When running CREATE FUNCTION namespace.function Spark fails the command if the current catalog is set to a v2 catalog.

### How was this patch tested?

Unit tests.

Closes #26890 from planga82/feature/SPARK-30039_CreateFunctionV2Command.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-08 00:38:15 +08:00
Kent Yao 8c121b0827 [SPARK-30431][SQL] Update SqlBase.g4 to create commentSpec pattern like locationSpec
### What changes were proposed in this pull request?

In `SqlBase.g4`, the `comment` clause is used as `COMMENT comment=STRING` and `COMMENT STRING` in many places.

While the `location` clause often appears along with the `comment` clause with a pattern defined as
```sql
locationSpec
    : LOCATION STRING
    ;
```
Then, we have to visit `locationSpec` as a `List` but comment as a single token.

We defined `commentSpec` for the comment clause to simplify and unify the grammar and the invocations.

### Why are the changes needed?

To simplify the grammar.

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

no
### How was this patch tested?

existing tests

Closes #27102 from yaooqinn/SPARK-30431.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 22:12:09 +08:00
Terry Kim 314e70fe23 [SPARK-30214][SQL] V2 commands resolves namespaces with new resolution framework
### What changes were proposed in this pull request?

#26847 introduced new framework for resolving catalog/namespaces. This PR proposes to integrate commands that need to resolve namespaces into the new framework.

### Why are the changes needed?

This is one of the work items for moving into the new resolution framework. Resolving v1/v2 tables with the new framework will be followed up in different PRs.

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

No

### How was this patch tested?

Existing tests should cover the changes.

Closes #27095 from imback82/unresolved_ns.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 21:32:08 +08:00
HyukjinKwon 866b7df348 [SPARK-30335][SQL][DOCS] Add a note first, last, collect_list and collect_set can be non-deterministic in SQL function docs as well
### What changes were proposed in this pull request?
This PR adds a note first and last can be non-deterministic in SQL function docs as well.
This is already documented in `functions.scala`.

### Why are the changes needed?
Some people look reading SQL docs only.

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

### How was this patch tested?
Jenkins will test.

Closes #27099 from HyukjinKwon/SPARK-30335.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-07 14:31:59 +09:00
Josh Rosen 7a1a5db35f [SPARK-30414][SQL] ParquetRowConverter optimizations: arrays, maps, plus misc. constant factors
### What changes were proposed in this pull request?

This PR implements multiple performance optimizations for `ParquetRowConverter`, achieving some modest constant-factor wins for all fields and larger wins for map and array fields:

- Add `private[this]` to several `val`s (90cebf080a5d3857ea8cf2a89e8e060b8b5a2fbf)
- Keep a `fieldUpdaters` array, saving two`.updater()` calls per field (7318785d350cc924198d7514e40973fd76d54ad5): I suspect that these are often megamorphic calls, so cutting these out seems like it could be a relatively large performance win.
- Only call `currentRow.numFields` once per `start()` call (e05de150813b639929c18af1df09ec718d2d16fc): previously we'd call it once per field and this had a significant enough cost that it was visible during profiling.
- Reuse buffers in array and map converters (c7d1534685fbad5d2280b082f37bed6d75848e76, 6d16f596ef6af9fd8946a062f79d0eeace9e1959): previously we would create a brand-new Scala `ArrayBuffer` for each field read, but this isn't actually necessary because the data is already copied into a fresh array when `end()` constructs a `GenericArrayData`.

### Why are the changes needed?

To improve Parquet read performance; this is complementary to #26993's (orthogonal) improvements for nested struct read performance.

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

No.

### How was this patch tested?

Existing tests, plus manual benchmarking with both synthetic and realistic schemas (similar to the ones in #26993). I've seen ~10%+ improvements in scan performance on certain real-world datasets.

Closes #27089 from JoshRosen/joshrosen/more-ParquetRowConverter-optimizations.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-07 14:30:10 +09:00
Josh Rosen 93d3ab88cd [SPARK-30338][SQL] Avoid unnecessary InternalRow copies in ParquetRowConverter
### What changes were proposed in this pull request?

This PR modifies `ParquetRowConverter` to remove unnecessary `InternalRow.copy()` calls for structs that are directly nested in other structs.

### Why are the changes needed?

These changes  can significantly improve performance when reading Parquet files that contain deeply-nested structs with many fields.

The `ParquetRowConverter` uses per-field `Converter`s for handling individual fields. Internally, these converters may have mutable state and may return mutable objects. In most cases, each `converter` is only invoked once per Parquet record (this is true for top-level fields, for example). However, arrays and maps may call their child element converters multiple times per Parquet record: in these cases we must be careful to copy any mutable outputs returned by child converters.

In the existing code, `InternalRow`s are copied whenever they are stored into _any_ parent container (not just maps and arrays). This copying can be especially expensive for deeply-nested fields, since a deep copy is performed at every level of nesting.

This PR modifies the code to avoid copies for structs that are directly nested in structs; see inline code comments for an argument for why this is safe.

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

No.

### How was this patch tested?

**Correctness**:  I added new test cases to `ParquetIOSuite` to increase coverage of nested structs, including structs nested in arrays: previously this suite didn't test that case, so we used to lack mutation coverage of this `copy()` code (the suite's tests still passed if I incorrectly removed the `.copy()` in all cases). I also added a test for maps with struct keys and modified the existing "map with struct values" test case include maps with two elements (since the incorrect omission of a `copy()` can only be detected if the map has multiple elements).

**Performance**: I put together a simple local benchmark demonstrating the performance problems:

First, construct a nested schema:

```scala
  case class Inner(
    f1: Int,
    f2: Long,
    f3: String,
    f4: Int,
    f5: Long,
    f6: String,
    f7: Int,
    f8: Long,
    f9: String,
    f10: Int
  )

  case class Wrapper1(inner: Inner)
  case class Wrapper2(wrapper1: Wrapper1)
  case class Wrapper3(wrapper2: Wrapper2)
```

`Wrapper3`'s schema looks like:

```
root
 |-- wrapper2: struct (nullable = true)
 |    |-- wrapper1: struct (nullable = true)
 |    |    |-- inner: struct (nullable = true)
 |    |    |    |-- f1: integer (nullable = true)
 |    |    |    |-- f2: long (nullable = true)
 |    |    |    |-- f3: string (nullable = true)
 |    |    |    |-- f4: integer (nullable = true)
 |    |    |    |-- f5: long (nullable = true)
 |    |    |    |-- f6: string (nullable = true)
 |    |    |    |-- f7: integer (nullable = true)
 |    |    |    |-- f8: long (nullable = true)
 |    |    |    |-- f9: string (nullable = true)
 |    |    |    |-- f10: integer (nullable = true)
```

Next, generate some fake data:

```scala
  val data = spark.range(1, 1000 * 1000 * 25, 1, 1).map { i =>
    Wrapper3(Wrapper2(Wrapper1(Inner(
      i.toInt,
      i * 2,
      (i * 3).toString,
      (i * 4).toInt,
      i * 5,
      (i * 6).toString,
      (i * 7).toInt,
      i * 8,
      (i * 9).toString,
      (i * 10).toInt
    ))))
  }

  data.write.mode("overwrite").parquet("/tmp/parquet-test")
```

I then ran a simple benchmark consisting of

```
spark.read.parquet("/tmp/parquet-test").selectExpr("hash(*)").rdd.count()
```

where the `hash(*)` is designed to force decoding of all Parquet fields but avoids `RowEncoder` costs in the `.rdd.count()` stage.

In the old code, expensive copying takes place at every level of nesting; this is apparent in the following flame graph:

![image](https://user-images.githubusercontent.com/50748/71389014-88a15380-25af-11ea-9537-3e87a2aef179.png)

After this PR's changes, the above toy benchmark runs ~30% faster.

Closes #26993 from JoshRosen/joshrosen/faster-parquet-nested-scan-by-avoiding-copies.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 13:01:37 +08:00
yi.wu da076153aa [SPARK-30433][SQL] Make conflict attributes resolution more scalable in ResolveReferences
### What changes were proposed in this pull request?

This PR tries to make conflict attributes resolution in `ResolveReferences` more scalable by doing resolution in batch way.

### Why are the changes needed?

Currently, `ResolveReferences` rule only resolves conflict attributes of one single conflict plan pair in one iteration, which can be inefficient when there're many conflicts.

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

No.

### How was this patch tested?

Covered by existed tests.

Closes #27105 from Ngone51/resolve-conflict-columns-in-batch.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 12:05:27 +08:00
Yuming Wang 17881a467a [SPARK-19784][SPARK-25403][SQL] Refresh the table even table stats is empty
## What changes were proposed in this pull request?

We invalidate table relation once table data is changed by [SPARK-21237](https://issues.apache.org/jira/browse/SPARK-21237). But there is a situation we have not invalidated(`spark.sql.statistics.size.autoUpdate.enabled=false` and `table.stats.isEmpty`):
07c4b9bd1f/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala (L44-L54)

This will introduce some issues, e.g. [SPARK-19784](https://issues.apache.org/jira/browse/SPARK-19784), [SPARK-19845](https://issues.apache.org/jira/browse/SPARK-19845), [SPARK-25403](https://issues.apache.org/jira/browse/SPARK-25403), [SPARK-25332](https://issues.apache.org/jira/browse/SPARK-25332) and [SPARK-28413](https://issues.apache.org/jira/browse/SPARK-28413).

This is a example to reproduce [SPARK-19784](https://issues.apache.org/jira/browse/SPARK-19784):
```scala
val path = "/tmp/spark/parquet"
spark.sql("CREATE TABLE t (a INT) USING parquet")
spark.sql("INSERT INTO TABLE t VALUES (1)")
spark.range(5).toDF("a").write.parquet(path)
spark.sql(s"ALTER TABLE t SET LOCATION '${path}'")
spark.table("t").count() // return 1
spark.sql("refresh table t")
spark.table("t").count() // return 5
```

This PR invalidates the table relation in this case(`spark.sql.statistics.size.autoUpdate.enabled=false` and `table.stats.isEmpty`) to fix this issue.

## How was this patch tested?

unit tests

Closes #22721 from wangyum/SPARK-25403.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-07 11:41:34 +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
angerszhu 3eade744f8 [SPARK-29800][SQL] Rewrite non-correlated EXISTS subquery use ScalaSubquery to optimize perf
### What changes were proposed in this pull request?

Current catalyst rewrite non-correlated exists subquery to BroadcastNestLoopJoin, it's performance is not good , now we rewrite non-correlated EXISTS subquery to ScalaSubquery to optimize the performance.
We rewrite
```
 WHERE EXISTS (SELECT A FROM TABLE B WHERE COL1 > 10)
```
to
```
 WHERE (SELECT 1 FROM (SELECT A FROM TABLE B WHERE COL1 > 10) LIMIT 1) IS NOT NULL
```
to avoid build join to solve EXISTS expression.

### Why are the changes needed?
Optimize EXISTS performance.

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

### How was this patch tested?
Manuel Tested

Closes #26437 from AngersZhuuuu/SPARK-29800.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-06 22:54:37 +08:00
root1 ebd2fd7e02 [SPARK-30415][SQL] Improve Readability of SQLConf Doc
### What changes were proposed in this pull request?
SQLCOnf Doc updated.

### Why are the changes needed?
Some doc comments were not written properly. Space was missing at many places. This patch updates the doc.

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

### How was this patch tested?
Documentation update.

Closes #27091 from iRakson/SQLConfDoc.

Authored-by: root1 <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-04 15:49:11 -06:00
Wenchen Fan be4faafee4 Revert "[SPARK-23264][SQL] Make INTERVAL keyword optional when ANSI enabled"
### What changes were proposed in this pull request?

Revert https://github.com/apache/spark/pull/20433 .
### Why are the changes needed?

According to the SQL standard, the INTERVAL prefix is required:
```
<interval literal> ::=
  INTERVAL [ <sign> ] <interval string> <interval qualifier>

<interval string> ::=
  <quote> <unquoted interval string> <quote>
```

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

yes, but omitting the INTERVAL prefix is a new feature in 3.0

### How was this patch tested?

existing tests

Closes #27080 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-01-03 12:51:10 -08:00
Yuming Wang 568ad4e77a [SPARK-29947][SQL] Improve ResolveRelations performance
### What changes were proposed in this pull request?

It is very common for a SQL query to query a table more than once. For example:
```
== Physical Plan ==
*(12) HashAggregate(keys=[cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367], functions=[sum(coalesce(bid_count#34, 0)), sum(coalesce(ck_trans_count#35, 0)), sum(coalesce(ended_bid_count#36, 0)), sum(coalesce(ended_lstg_count#37, 0)), sum(coalesce(ended_success_lstg_count#38, 0)), sum(coalesce(item_sold_count#39, 0)), sum(coalesce(new_lstg_count#40, 0)), sum(coalesce(gmv_us_amt#41, 0.00)), sum(coalesce(gmv_slr_lc_amt#42, 0.00)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_insrtn_fee_us_amt#46, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_insrtn_crd_us_amt#50, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fetr_fee_us_amt#54, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fetr_crd_us_amt#58, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fv_fee_us_amt#62, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fv_crd_us_amt#67, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_l_fee_us_amt#72, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_l_crd_us_amt#76, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_nl_fee_us_amt#80, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_nl_crd_us_amt#84, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_slr_tools_fee_us_amt#88, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_slr_tools_crd_us_amt#92, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), sum(coalesce(rvnu_unasgnd_us_amt#96, 0.000000)), sum((coalesce(rvnu_transaction_us_amt#112, 0.0) + coalesce(rvnu_transaction_crd_us_amt#115, 0.0))), sum((coalesce(rvnu_total_us_amt#118, 0.0) + coalesce(rvnu_total_crd_us_amt#121, 0.0)))])
+- Exchange hashpartitioning(cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367, 200), true, [id=#403]
   +- *(11) HashAggregate(keys=[cmn_mtrc_summ_dt#21, rev_rollup#1279, CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END AS CASE WHEN (rev_rollup#1319 = rev_rollup#1279) THEN 0 ELSE 1 END#1366, CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END AS CASE WHEN cast(sap_category_id#24 as decimal(10,0)) IN (5,7,23,41) THEN 0 ELSE 1 END#1367], functions=[partial_sum(coalesce(bid_count#34, 0)), partial_sum(coalesce(ck_trans_count#35, 0)), partial_sum(coalesce(ended_bid_count#36, 0)), partial_sum(coalesce(ended_lstg_count#37, 0)), partial_sum(coalesce(ended_success_lstg_count#38, 0)), partial_sum(coalesce(item_sold_count#39, 0)), partial_sum(coalesce(new_lstg_count#40, 0)), partial_sum(coalesce(gmv_us_amt#41, 0.00)), partial_sum(coalesce(gmv_slr_lc_amt#42, 0.00)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_insrtn_fee_us_amt#46, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_insrtn_crd_us_amt#50, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fetr_fee_us_amt#54, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fetr_crd_us_amt#58, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_fv_fee_us_amt#62, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_fv_crd_us_amt#67, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_l_fee_us_amt#72, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_l_crd_us_amt#76, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_othr_nl_fee_us_amt#80, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_othr_nl_crd_us_amt#84, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(CheckOverflow((promote_precision(cast(coalesce(rvnu_slr_tools_fee_us_amt#88, 0.000000) as decimal(19,6))) + promote_precision(cast(coalesce(rvnu_slr_tools_crd_us_amt#92, 0.000000) as decimal(19,6)))), DecimalType(19,6), true)), partial_sum(coalesce(rvnu_unasgnd_us_amt#96, 0.000000)), partial_sum((coalesce(rvnu_transaction_us_amt#112, 0.0) + coalesce(rvnu_transaction_crd_us_amt#115, 0.0))), partial_sum((coalesce(rvnu_total_us_amt#118, 0.0) + coalesce(rvnu_total_crd_us_amt#121, 0.0)))])
      +- *(11) Project [cmn_mtrc_summ_dt#21, sap_category_id#24, bid_count#34, ck_trans_count#35, ended_bid_count#36, ended_lstg_count#37, ended_success_lstg_count#38, item_sold_count#39, new_lstg_count#40, gmv_us_amt#41, gmv_slr_lc_amt#42, rvnu_insrtn_fee_us_amt#46, rvnu_insrtn_crd_us_amt#50, rvnu_fetr_fee_us_amt#54, rvnu_fetr_crd_us_amt#58, rvnu_fv_fee_us_amt#62, rvnu_fv_crd_us_amt#67, rvnu_othr_l_fee_us_amt#72, rvnu_othr_l_crd_us_amt#76, rvnu_othr_nl_fee_us_amt#80, rvnu_othr_nl_crd_us_amt#84, rvnu_slr_tools_fee_us_amt#88, rvnu_slr_tools_crd_us_amt#92, rvnu_unasgnd_us_amt#96, ... 6 more fields]
         +- *(11) BroadcastHashJoin [byr_cntry_id#23], [cntry_id#1309], LeftOuter, BuildRight
            :- *(11) Project [cmn_mtrc_summ_dt#21, byr_cntry_id#23, sap_category_id#24, bid_count#34, ck_trans_count#35, ended_bid_count#36, ended_lstg_count#37, ended_success_lstg_count#38, item_sold_count#39, new_lstg_count#40, gmv_us_amt#41, gmv_slr_lc_amt#42, rvnu_insrtn_fee_us_amt#46, rvnu_insrtn_crd_us_amt#50, rvnu_fetr_fee_us_amt#54, rvnu_fetr_crd_us_amt#58, rvnu_fv_fee_us_amt#62, rvnu_fv_crd_us_amt#67, rvnu_othr_l_fee_us_amt#72, rvnu_othr_l_crd_us_amt#76, rvnu_othr_nl_fee_us_amt#80, rvnu_othr_nl_crd_us_amt#84, rvnu_slr_tools_fee_us_amt#88, rvnu_slr_tools_crd_us_amt#92, ... 6 more fields]
            :  +- *(11) BroadcastHashJoin [slr_cntry_id#28], [cntry_id#1269], LeftOuter, BuildRight
            :     :- *(11) Project [gen_attr_1#360 AS cmn_mtrc_summ_dt#21, gen_attr_5#267 AS byr_cntry_id#23, gen_attr_7#268 AS sap_category_id#24, gen_attr_15#272 AS slr_cntry_id#28, gen_attr_27#278 AS bid_count#34, gen_attr_29#279 AS ck_trans_count#35, gen_attr_31#280 AS ended_bid_count#36, gen_attr_33#282 AS ended_lstg_count#37, gen_attr_35#283 AS ended_success_lstg_count#38, gen_attr_37#284 AS item_sold_count#39, gen_attr_39#281 AS new_lstg_count#40, gen_attr_41#285 AS gmv_us_amt#41, gen_attr_43#287 AS gmv_slr_lc_amt#42, gen_attr_51#290 AS rvnu_insrtn_fee_us_amt#46, gen_attr_59#294 AS rvnu_insrtn_crd_us_amt#50, gen_attr_67#298 AS rvnu_fetr_fee_us_amt#54, gen_attr_75#302 AS rvnu_fetr_crd_us_amt#58, gen_attr_83#306 AS rvnu_fv_fee_us_amt#62, gen_attr_93#311 AS rvnu_fv_crd_us_amt#67, gen_attr_103#316 AS rvnu_othr_l_fee_us_amt#72, gen_attr_111#320 AS rvnu_othr_l_crd_us_amt#76, gen_attr_119#324 AS rvnu_othr_nl_fee_us_amt#80, gen_attr_127#328 AS rvnu_othr_nl_crd_us_amt#84, gen_attr_135#332 AS rvnu_slr_tools_fee_us_amt#88, ... 6 more fields]
            :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_308#777 as decimal(20,0))], [cast(gen_attr_309#803 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_310#674 as int)], [cast(gen_attr_311#774 as int)], LeftOuter, BuildRight
            :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_5#267 as decimal(20,0))], [cast(gen_attr_312#665 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 5 more fields]
            :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_313#565 as decimal(20,0))], [cast(gen_attr_314#591 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_315#462 as int)], [cast(gen_attr_316#562 as int)], LeftOuter, BuildRight
            :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 6 more fields]
            :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_15#272 as decimal(20,0))], [cast(gen_attr_317#453 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, gen_attr_143#336, ... 5 more fields]
            :     :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_25#277 as decimal(20,0))], [cast(gen_attr_318#379 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :     :- *(11) Project [gen_attr_5#267, gen_attr_7#268, gen_attr_15#272, gen_attr_25#277, gen_attr_27#278, gen_attr_29#279, gen_attr_31#280, gen_attr_39#281, gen_attr_33#282, gen_attr_35#283, gen_attr_37#284, gen_attr_41#285, gen_attr_43#287, gen_attr_51#290, gen_attr_59#294, gen_attr_67#298, gen_attr_75#302, gen_attr_83#306, gen_attr_93#311, gen_attr_103#316, gen_attr_111#320, gen_attr_119#324, gen_attr_127#328, gen_attr_135#332, ... 6 more fields]
            :     :     :     :     :     :     :     :     :  +- *(11) BroadcastHashJoin [cast(gen_attr_23#276 as decimal(20,0))], [cast(gen_attr_319#367 as decimal(20,0))], LeftOuter, BuildRight
            :     :     :     :     :     :     :     :     :     :- *(11) Project [byr_cntry_id#1169 AS gen_attr_5#267, sap_category_id#1170 AS gen_attr_7#268, slr_cntry_id#1174 AS gen_attr_15#272, lstg_curncy_id#1178 AS gen_attr_23#276, blng_curncy_id#1179 AS gen_attr_25#277, bid_count#1180 AS gen_attr_27#278, ck_trans_count#1181 AS gen_attr_29#279, ended_bid_count#1182 AS gen_attr_31#280, new_lstg_count#1183 AS gen_attr_39#281, ended_lstg_count#1184 AS gen_attr_33#282, ended_success_lstg_count#1185 AS gen_attr_35#283, item_sold_count#1186 AS gen_attr_37#284, gmv_us_amt#1187 AS gen_attr_41#285, gmv_slr_lc_amt#1189 AS gen_attr_43#287, rvnu_insrtn_fee_us_amt#1192 AS gen_attr_51#290, rvnu_insrtn_crd_us_amt#1196 AS gen_attr_59#294, rvnu_fetr_fee_us_amt#1200 AS gen_attr_67#298, rvnu_fetr_crd_us_amt#1204 AS gen_attr_75#302, rvnu_fv_fee_us_amt#1208 AS gen_attr_83#306, rvnu_fv_crd_us_amt#1213 AS gen_attr_93#311, rvnu_othr_l_fee_us_amt#1218 AS gen_attr_103#316, rvnu_othr_l_crd_us_amt#1222 AS gen_attr_111#320, rvnu_othr_nl_fee_us_amt#1226 AS gen_attr_119#324, rvnu_othr_nl_crd_us_amt#1230 AS gen_attr_127#328, ... 7 more fields]
            :     :     :     :     :     :     :     :     :     :  +- *(11) ColumnarToRow
            :     :     :     :     :     :     :     :     :     :     +- FileScan parquet default.big_table1[byr_cntry_id#1169,sap_category_id#1170,slr_cntry_id#1174,lstg_curncy_id#1178,blng_curncy_id#1179,bid_count#1180,ck_trans_count#1181,ended_bid_count#1182,new_lstg_count#1183,ended_lstg_count#1184,ended_success_lstg_count#1185,item_sold_count#1186,gmv_us_amt#1187,gmv_slr_lc_amt#1189,rvnu_insrtn_fee_us_amt#1192,rvnu_insrtn_crd_us_amt#1196,rvnu_fetr_fee_us_amt#1200,rvnu_fetr_crd_us_amt#1204,rvnu_fv_fee_us_amt#1208,rvnu_fv_crd_us_amt#1213,rvnu_othr_l_fee_us_amt#1218,rvnu_othr_l_crd_us_amt#1222,rvnu_othr_nl_fee_us_amt#1226,rvnu_othr_nl_crd_us_amt#1230,... 7 more fields] Batched: true, DataFilters: [], Format: Parquet, Location: PrunedInMemoryFileIndex[], PartitionFilters: [isnotnull(cmn_mtrc_summ_dt#1262), (cmn_mtrc_summ_dt#1262 >= 18078), (cmn_mtrc_summ_dt#1262 <= 18..., PushedFilters: [], ReadSchema: struct<byr_cntry_id:decimal(4,0),sap_category_id:decimal(9,0),slr_cntry_id:decimal(4,0),lstg_curn...
            :     :     :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#288]
            :     :     :     :     :     :     :     :     :        +- *(1) Project [CURNCY_ID#1263 AS gen_attr_319#367]
            :     :     :     :     :     :     :     :     :           +- *(1) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :     :     :     :     :              +- *(1) ColumnarToRow
            :     :     :     :     :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#297]
            :     :     :     :     :     :     :     :        +- *(2) Project [CURNCY_ID#1263 AS gen_attr_318#379]
            :     :     :     :     :     :     :     :           +- *(2) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :     :     :     :              +- *(2) ColumnarToRow
            :     :     :     :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(4,0), true] as decimal(20,0)))), [id=#306]
            :     :     :     :     :     :     :        +- *(3) Project [cntry_id#1269 AS gen_attr_317#453, rev_rollup_id#1278 AS gen_attr_315#462]
            :     :     :     :     :     :     :           +- *(3) Filter isnotnull(cntry_id#1269)
            :     :     :     :     :     :     :              +- *(3) ColumnarToRow
            :     :     :     :     :     :     :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup_id#1278] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup_id:smallint>
            :     :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(cast(input[0, smallint, true] as int) as bigint))), [id=#315]
            :     :     :     :     :     :        +- *(4) Project [rev_rollup_id#1286 AS gen_attr_316#562, curncy_id#1289 AS gen_attr_313#565]
            :     :     :     :     :     :           +- *(4) Filter isnotnull(rev_rollup_id#1286)
            :     :     :     :     :     :              +- *(4) ColumnarToRow
            :     :     :     :     :     :                 +- FileScan parquet default.small_table3[rev_rollup_id#1286,curncy_id#1289] Batched: true, DataFilters: [isnotnull(rev_rollup_id#1286)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table3], PartitionFilters: [], PushedFilters: [IsNotNull(rev_rollup_id)], ReadSchema: struct<rev_rollup_id:smallint,curncy_id:decimal(4,0)>
            :     :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#324]
            :     :     :     :     :        +- *(5) Project [CURNCY_ID#1263 AS gen_attr_314#591]
            :     :     :     :     :           +- *(5) Filter isnotnull(CURNCY_ID#1263)
            :     :     :     :     :              +- *(5) ColumnarToRow
            :     :     :     :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(4,0), true] as decimal(20,0)))), [id=#333]
            :     :     :     :        +- *(6) Project [cntry_id#1269 AS gen_attr_312#665, rev_rollup_id#1278 AS gen_attr_310#674]
            :     :     :     :           +- *(6) Filter isnotnull(cntry_id#1269)
            :     :     :     :              +- *(6) ColumnarToRow
            :     :     :     :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup_id#1278] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup_id:smallint>
            :     :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(cast(input[0, smallint, true] as int) as bigint))), [id=#342]
            :     :     :        +- *(7) Project [rev_rollup_id#1286 AS gen_attr_311#774, curncy_id#1289 AS gen_attr_308#777]
            :     :     :           +- *(7) Filter isnotnull(rev_rollup_id#1286)
            :     :     :              +- *(7) ColumnarToRow
            :     :     :                 +- FileScan parquet default.small_table3[rev_rollup_id#1286,curncy_id#1289] Batched: true, DataFilters: [isnotnull(rev_rollup_id#1286)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table3], PartitionFilters: [], PushedFilters: [IsNotNull(rev_rollup_id)], ReadSchema: struct<rev_rollup_id:smallint,curncy_id:decimal(4,0)>
            :     :     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, decimal(9,0), true] as decimal(20,0)))), [id=#351]
            :     :        +- *(8) Project [CURNCY_ID#1263 AS gen_attr_309#803]
            :     :           +- *(8) Filter isnotnull(CURNCY_ID#1263)
            :     :              +- *(8) ColumnarToRow
            :     :                 +- FileScan parquet default.small_table1[CURNCY_ID#1263] Batched: true, DataFilters: [isnotnull(CURNCY_ID#1263)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table1], PartitionFilters: [], PushedFilters: [IsNotNull(CURNCY_ID)], ReadSchema: struct<CURNCY_ID:decimal(9,0)>, SelectedBucketsCount: 1 out of 1
            :     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, decimal(4,0), true])), [id=#360]
            :        +- *(9) Project [cntry_id#1269, rev_rollup#1279]
            :           +- *(9) Filter isnotnull(cntry_id#1269)
            :              +- *(9) ColumnarToRow
            :                 +- FileScan parquet default.small_table2[cntry_id#1269,rev_rollup#1279] Batched: true, DataFilters: [isnotnull(cntry_id#1269)], Format: Parquet, Location: InMemoryFileIndex[file:/user/hive/warehouse/small_table2], PartitionFilters: [], PushedFilters: [IsNotNull(cntry_id)], ReadSchema: struct<cntry_id:decimal(4,0),rev_rollup:string>
            +- ReusedExchange [cntry_id#1309, rev_rollup#1319], BroadcastExchange HashedRelationBroadcastMode(List(input[0, decimal(4,0), true])), [id=#360]
```
This PR try to improve `ResolveTables` and `ResolveRelations` performance by reducing the connection times to Hive Metastore Server in such case.

### Why are the changes needed?
1. Reduce the connection times to Hive Metastore Server.
2. Improve `ResolveTables` and `ResolveRelations` performance.

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

### How was this patch tested?

manual test.
After [SPARK-29606](https://issues.apache.org/jira/browse/SPARK-29606) and before this PR:
```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 9323
Total time: 2.687441263 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   929173767 / 930133504                           2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables                                      0 / 383363402                                   0 / 18
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         0 / 99433540                                    0 / 4
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            41809394 / 83727901                             2 / 18
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               71372977 / 71372977                             1 / 1
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              0 / 59071933                                    0 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  37858325 / 58471776                             5 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 20889892 / 53229016                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     23428968 / 50890815                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   23230666 / 49182607                             1 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 43638350                                    0 / 18
org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              17194844 / 42530885                             1 / 6
```
After [SPARK-29606](https://issues.apache.org/jira/browse/SPARK-29606) and after this PR:
```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 9323
Total time: 2.163765869 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   658905353 / 659829383                           2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables                                      0 / 220708715                                   0 / 18
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         0 / 99606816                                    0 / 4
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            39616060 / 78215752                             2 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  36706549 / 54917789                             5 / 18
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               53561921 / 53561921                             1 / 1
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              0 / 52329678                                    0 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 20945755 / 49695998                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     20872241 / 46740145                             1 / 18
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   19780298 / 44327227                             1 / 18
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 42312023                                    0 / 18
org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              17197393 / 39501424                             1 / 6
```

Closes #26589 from wangyum/SPARK-29947.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 22:36:31 +08:00
yi.wu e38964c442 [SPARK-29768][SQL][FOLLOW-UP] Improve handling non-deterministic filter of ScanOperation
### What changes were proposed in this pull request?

1. For `ScanOperation`, if it collects more than one filters, then all filters must be deterministic. And filter can be non-deterministic iff there's only one collected filter.

2. `FileSourceStrategy` should filter out non-deterministic filter, as it will hit haven't initialized exception if it's a partition related filter.

### Why are the changes needed?

Strictly follow `CombineFilters`'s behavior which doesn't allow combine two filters where non-deterministic predicates exist. And avoid hitting exception for file source.

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

No

### How was this patch tested?

Test exists.

Closes #27073 from Ngone51/SPARK-29768-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 21:48:14 +08:00
Kent Yao c49388a484 [SPARK-30214][SQL] A new framework to resolve v2 commands
### What changes were proposed in this pull request?
Currently, we have a v2 adapter for v1 catalog (`V2SessionCatalog`), all the table/namespace commands can be implemented via v2 APIs.

Usually, a command needs to know which catalog it needs to operate, but different commands have different requirements about what to resolve. A few examples:

  - `DROP NAMESPACE`: only need to know the name of the namespace.
  - `DESC NAMESPACE`: need to lookup the namespace and get metadata, but is done during execution
  - `DROP TABLE`: need to do lookup and make sure it's a table not (temp) view.
  - `DESC TABLE`: need to lookup the table and get metadata.

For namespaces, the analyzer only needs to find the catalog and the namespace name. The command can do lookup during execution if needed.

For tables, mostly commands need the analyzer to do lookup.

Note that, table and namespace have a difference: `DESC NAMESPACE testcat` works and describes the root namespace under `testcat`, while `DESC TABLE testcat` fails if there is no table `testcat` under the current catalog. It's because namespaces can be named [], but tables can't. The commands should explicitly specify it needs to operate on namespace or table.

In this Pull Request, we introduce a new framework to resolve v2 commands:
1. parser creates logical plans or commands with `UnresolvedNamespace`/`UnresolvedTable`/`UnresolvedView`/`UnresolvedRelation`. (CREATE TABLE still keeps Seq[String], as it doesn't need to look up relations)
2. analyzer converts
2.1 `UnresolvedNamespace` to `ResolvesNamespace` (contains catalog and namespace identifier)
2.2 `UnresolvedTable` to `ResolvedTable` (contains catalog, identifier and `Table`)
2.3 `UnresolvedView` to `ResolvedView` (will be added later when we migrate view commands)
2.4 `UnresolvedRelation` to relation.
3. an extra analyzer rule to match commands with `V1Table` and converts them to corresponding v1 commands. This will be added later when we migrate existing commands
4. planner matches commands and converts them to the corresponding physical nodes.

We also introduce brand new v2 commands - the `comment` syntaxes to illustrate how to work with the newly added framework.
```sql
COMMENT ON (DATABASE|SCHEMA|NAMESPACE) ... IS ...
COMMENT ON TABLE ... IS ...
```
Details about the `comment` syntaxes:
As the new design of catalog v2, some properties become reserved, e.g. `location`, `comment`. We are going to disable setting reserved properties by dbproperties or tblproperites directly to avoid confliction with their related subClause or specific commands.

They are the best practices from PostgreSQL and presto.

https://www.postgresql.org/docs/12/sql-comment.html
https://prestosql.io/docs/current/sql/comment.html

Mostly, the basic thoughts of the new framework came from the discussions bellow with cloud-fan,  https://github.com/apache/spark/pull/26847#issuecomment-564510061,

### Why are the changes needed?
To make it easier to add new v2 commands, and easier to unify the table relation behavior.

### Does this PR introduce any user-facing change?
yes, add new syntax

### How was this patch tested?

add uts.

Closes #26847 from yaooqinn/SPARK-30214.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 16:09:06 +08:00
Maxim Gekk 51373467cc [SPARK-30412][SQL][TESTS] Eliminate warnings in Java tests regarding to deprecated Spark SQL API
### What changes were proposed in this pull request?
In the PR, I propose to add the `SuppressWarnings("deprecation")` annotation to Java tests for deprecated Spark SQL APIs.

### Why are the changes needed?
This eliminates the following warnings:
```
sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java
    Warning:Warning:line (32)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (91)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (100)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (109)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (118)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated

sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java
    Warning:Warning:line (28)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (37)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (46)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (55)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated
    Warning:Warning:line (64)java: org.apache.spark.sql.expressions.javalang.typed in org.apache.spark.sql.expressions.javalang has been deprecated

sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
    Warning:Warning:line (478)java: json(org.apache.spark.api.java.JavaRDD<java.lang.String>) in org.apache.spark.sql.DataFrameReader has been deprecated
```
and highlights warnings about real problems.

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

### How was this patch tested?
By existing test suites `Java8DatasetAggregatorSuite.java`, `JavaDataFrameSuite.java` and `JavaDatasetAggregatorSuite.java`.

Closes #27081 from MaxGekk/eliminate-warnings-part2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-03 13:26:48 +09:00
Maxim Gekk a469976e6e [SPARK-29930][SQL][FOLLOW-UP] Allow only default value to be set for removed SQL configs
### What changes were proposed in this pull request?
In the PR, I propose to throw `AnalysisException` when a removed SQL config is set to non-default value. The following SQL configs removed by #26559 are marked as removed:
1. `spark.sql.fromJsonForceNullableSchema`
2. `spark.sql.legacy.compareDateTimestampInTimestamp`
3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation`

### Why are the changes needed?
To improve user experience with Spark SQL by notifying of removed SQL configs used by users.

### Does this PR introduce any user-facing change?
Yes, before the `set` command was silently ignored:
```sql
spark-sql> set spark.sql.fromJsonForceNullableSchema=false;
spark.sql.fromJsonForceNullableSchema	false
```
after the exception should be raised:
```sql
spark-sql> set spark.sql.fromJsonForceNullableSchema=false;
Error in query: The SQL config 'spark.sql.fromJsonForceNullableSchema' was removed in the version 3.0.0. It was removed to prevent errors like SPARK-23173 for non-default value.;
```

### How was this patch tested?
Added new tests into `SQLConfSuite` for both cases when removed SQL configs are set to default and non-default values.

Closes #27057 from MaxGekk/remove-sql-configs-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-03 10:41:30 +09:00
Kent Yao e04309cb1f [SPARK-30341][SQL] Overflow check for interval arithmetic operations
### What changes were proposed in this pull request?

1. For the interval arithmetic functions, e.g. `add`/`subtract`/`negative`/`multiply`/`divide`, enable overflow check when `ANSI` is on.

2. For `multiply`/`divide`,  throw an exception when an overflow happens in spite of `ANSI` is on/off.

3. `add`/`subtract`/`negative` stay the same for backward compatibility.

4. `divide` by 0 throws ArithmeticException whether `ANSI` or not as same as numerics.

5. These behaviors fit the numeric type operations fully when ANSI is on.

6. These behaviors fit the numeric type operations fully when ANSI is off, except 2 and 4.

### Why are the changes needed?

1. bug fix
2. `ANSI` support

### Does this PR introduce any user-facing change?
When `ANSI` is on, interval `add`/`subtract`/`negative`/`multiply`/`divide` will overflow if any field overflows

### How was this patch tested?

add unit tests

Closes #26995 from yaooqinn/SPARK-30341.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 02:04:20 +08:00
Wenchen Fan 68260f5297 revert [SPARK-29680][SQL] Remove ALTER TABLE CHANGE COLUMN syntax
### What changes were proposed in this pull request?

Revert https://github.com/apache/spark/pull/26338 , as the syntax is actually the [hive style ALTER COLUMN](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment).

This PR brings it back, and make it support multi-catalog:
1. renaming is not allowed as `AlterTableAlterColumnStatement` can't do renaming.
2. column name should be multi-part

### Why are the changes needed?

to not break hive compatibility.

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

no, as the removal was merged in 3.0.

### How was this patch tested?

new parser tests

Closes #27076 from cloud-fan/alter.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 01:55:38 +08:00
Wenchen Fan 1743d5be7f [SPARK-30284][SQL] CREATE VIEW should keep the current catalog and namespace
### What changes were proposed in this pull request?

Update CREATE VIEW command to store the current catalog and namespace instead of current database in view metadata. Also update analyzer to leverage the catalog and namespace in view metastore to resolve relations inside views.

Note that, this PR still keeps the way we resolve views, by recursively calling Analyzer. This is necessary because view text may contain CTE, window spec, etc. which needs rules outside of the main resolution batch (e.g. `CTESubstitution`)

### Why are the changes needed?

To resolve relations inside view correctly.

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

Yes, fix a bug. Now tables referred by a view can be resolved correctly even if the current catalog/namespace has been updated.

### How was this patch tested?

a new test

Closes #26923 from cloud-fan/view.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 01:41:32 +08:00
jiake 6bd5494f34 [SPARK-30403][SQL] fix the NoSuchElementException when enable AQE with InSubquery expression
### What changes were proposed in this pull request?
This PR aim to fix the NoSuchElementException  exception when enable AQE with insubquery expression.

### Why are the changes needed?
Fix exception

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

### How was this patch tested?
added new ut

Closes #27068 from JkSelf/fixSubqueryIssue.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 23:11:56 +08:00
jiake 05f7b57ddc [SPARK-30407][SQL] fix the reset metric issue when enable AQE
### What changes were proposed in this pull request?
When working on [PR#26813](https://github.com/apache/spark/pull/26813), we encounter the exception in [here(the number of metrics(1) is 2 not 1 )](5d870ef0bc/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala (L120)). This PR fix the above exception.

### Why are the changes needed?
Fix exception

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

### How was this patch tested?
[this test with enable AQE](5d870ef0bc/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala (L120))

Closes #27074 from JkSelf/resetMetricsIssue.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 21:55:36 +08:00
Jungtaek Lim (HeartSaVioR) 5d870ef0bc [SPARK-26560][SQL] Spark should be able to run Hive UDF using jar regardless of current thread context classloader
### What changes were proposed in this pull request?

This patch is based on #23921 but revised to be simpler, as well as adds UT to test the behavior.
(This patch contains the commit from #23921 to retain credit.)

Spark loads new JARs for `ADD JAR` and `CREATE FUNCTION ... USING JAR` into jar classloader in shared state, and changes current thread's context classloader to jar classloader as many parts of remaining codes rely on current thread's context classloader.

This would work if the further queries will run in same thread and there's no change on context classloader for the thread, but once the context classloader of current thread is switched back by various reason, Spark fails to create instance of class for the function.

This bug mostly affects spark-shell, as spark-shell will roll back current thread's context classloader at every prompt. But it may also affects the case of job-server, where the queries may be running in multiple threads.

This patch fixes the issue via switching the context classloader to the classloader which loads the class. Hopefully FunctionBuilder created by `makeFunctionBuilder` has the information of Class as a part of closure, hence the Class itself can be provided regardless of current thread's context classloader.

### Why are the changes needed?

Without this patch, end users cannot execute Hive UDF using JAR twice in spark-shell.

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

No.

### How was this patch tested?

New UT.

Closes #27025 from HeartSaVioR/SPARK-26560-revised.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: nivo091 <nivedeeta.singh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-02 15:44:45 +08:00
yi.wu 83d289eef4 [SPARK-27638][SQL][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Change config name from `spark.sql.legacy.typeCoercion.datetimeToString` to `spark.sql.legacy.typeCoercion.datetimeToString.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins

Closes #27065 from Ngone51/SPARK-27638-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 15:35:33 +09:00
yi.wu 90794b617c [SPARK-27871][SQL][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Change config name from `spark.sql.optimizer.reassignLambdaVariableID` to `spark.sql.optimizer.reassignLambdaVariableID.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27063 from Ngone51/SPARK-27871-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 09:59:15 +09:00
Maxim Gekk b316d37365 [SPARK-30401][SQL] Call requireNonStaticConf() only once in set()
### What changes were proposed in this pull request?
Calls of `requireNonStaticConf()` are removed from the `set()` methods in RuntimeConfig because those methods invoke `def set(key: String, value: String): Unit` where `requireNonStaticConf()` is called as well.

### Why are the changes needed?
To avoid unnecessary calls of `requireNonStaticConf()`.

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

### How was this patch tested?
By existing tests from `SQLConfSuite`

Closes #27062 from MaxGekk/call-requireNonStaticConf-once.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-02 09:56:50 +09:00
Zhenhua Wang a8bf5d823b [SPARK-30339][SQL] Avoid to fail twice in function lookup
### What changes were proposed in this pull request?

Currently if function lookup fails, spark will give it a second change by casting decimal type to double type. But for cases where decimal type doesn't exist, it's meaningless to lookup again and causes extra cost like unnecessary metastore access. We should throw exceptions directly in these cases.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #26994 from wzhfy/avoid_udf_fail_twice.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:09:51 +09:00
Jungtaek Lim (HeartSaVioR) e054a0af6f [SPARK-29348][SQL][FOLLOWUP] Fix slight bug on streaming example for Dataset.observe
### What changes were proposed in this pull request?

This patch fixes a small bug in the example of streaming query, as the type of observable metrics is Java Map instead of Scala Map, so to use foreach it should be converted first.

### Why are the changes needed?

Described above.

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

No.

### How was this patch tested?

Ran below query via `spark-shell`:

**Streaming**

```scala
import scala.collection.JavaConverters._
import scala.util.Random
import org.apache.spark.sql.streaming.StreamingQueryListener
import org.apache.spark.sql.streaming.StreamingQueryListener._

spark.streams.addListener(new StreamingQueryListener() {
  override def onQueryProgress(event: QueryProgressEvent): Unit = {
    event.progress.observedMetrics.asScala.get("my_event").foreach { row =>
      // Trigger if the number of errors exceeds 5 percent
      val num_rows = row.getAs[Long]("rc")
      val num_error_rows = row.getAs[Long]("erc")
      val ratio = num_error_rows.toDouble / num_rows
      if (ratio > 0.05) {
        // Trigger alert
        println(s"alert! error ratio: $ratio")
      }
    }
  }

  def onQueryStarted(event: QueryStartedEvent): Unit = {}
  def onQueryTerminated(event: QueryTerminatedEvent): Unit = {}
})

val rates = spark
  .readStream
  .format("rate")
  .option("rowsPerSecond", 10)
  .load

val rand = new Random()
val df = rates.map { row => (row.getLong(1), if (row.getLong(1) % 2 == 0) "error" else null) }.toDF
val ds = df.selectExpr("_1 AS id", "_2 AS error")
// Observe row count (rc) and error row count (erc) in the batch Dataset
val observed_ds = ds.observe("my_event", count(lit(1)).as("rc"), count($"error").as("erc"))
observed_ds.writeStream.format("console").start()
```

Closes #27046 from HeartSaVioR/SPARK-29348-FOLLOWUP.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:08:25 +09:00
HyukjinKwon 7079e871a7 [SPARK-30185][SQL] Implement Dataset.tail API
### What changes were proposed in this pull request?

This PR proposes a `tail` API.

Namely, as below:

```scala
scala> spark.range(10).head(5)
res1: Array[Long] = Array(0, 1, 2, 3, 4)
scala> spark.range(10).tail(5)
res2: Array[Long] = Array(5, 6, 7, 8, 9)
```

Implementation details will be similar with `head` but it will be reversed:

1. Run the job against the last partition and collect rows. If this is enough, return as is.
2. If this is not enough, calculate the number of partitions to select more based upon
 `spark.sql.limit.scaleUpFactor`
3. Run more jobs against more partitions (in a reversed order compared to head) as many as the number calculated from 2.
4. Go to 2.

**Note that**, we don't guarantee the natural order in DataFrame in general - there are cases when it's deterministic and when it's not. We probably should write down this as a caveat separately.

### Why are the changes needed?

Many other systems support the way to take data from the end, for instance, pandas[1] and
 Python[2][3]. Scala collections APIs also have head and tail

On the other hand, in Spark, we only provide a way to take data from the start
 (e.g., DataFrame.head).

This has been requested multiple times here and there in Spark user mailing list[4], StackOverFlow[5][6], JIRA[7] and other third party projects such as
 Koalas[8]. In addition, this missing API seems explicitly mentioned in comparison to another system[9] time to time.

It seems we're missing non-trivial use case in Spark and this motivated me to propose this API.

[1] https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.tail.html?highlight=tail#pandas.DataFrame.tail
[2] https://stackoverflow.com/questions/10532473/head-and-tail-in-one-line
[3] https://stackoverflow.com/questions/646644/how-to-get-last-items-of-a-list-in-python
[4] http://apache-spark-user-list.1001560.n3.nabble.com/RDD-tail-td4217.html
[5] https://stackoverflow.com/questions/39544796/how-to-select-last-row-and-also-how-to-access-pyspark-dataframe-by-index
[6] https://stackoverflow.com/questions/45406762/how-to-get-the-last-row-from-dataframe
[7] https://issues.apache.org/jira/browse/SPARK-26433
[8] https://github.com/databricks/koalas/issues/343
[9] https://medium.com/chris_bour/6-differences-between-pandas-and-spark-dataframes-1380cec394d2

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

No, (new API)

### How was this patch tested?

Unit tests were added and manually tested.

Closes #26809 from HyukjinKwon/wip-tail.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 01:07:09 +09:00
Terry Kim a90ad5bf2a [SPARK-30370][SQL] Update SqlBase.g4 to combine namespace and database tokens
### What changes were proposed in this pull request?

In `SqlBase.g4`, `database` is defined as
```
database : DATABASE | SCHEMA;
```
and it is being used as `(database | NAMESPACE)` in many places.

This PR proposes to define the following and use it as discussed in https://github.com/apache/spark/pull/26847/files#r359754778:
```
namespace : NAMESPACE | DATABASE | SCHEMA;
```

### Why are the changes needed?

To simplify the grammar.

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

No.

### How was this patch tested?

There is no change in the actual grammar, so the existing tests should be sufficient.

Closes #27027 from imback82/sqlbase_namespace.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-30 14:58:02 +08:00
Xiao Li 919d551ddb Revert "[SPARK-29390][SQL] Add the justify_days(), justify_hours() and justif_interval() functions"
This reverts commit f926809a1f.

Closes #27032 from gatorsmile/revertSPARK-29390.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-12-29 15:25:14 -08:00
sandeep katta 16e5e79877 [SPARK-28670][SQL] create function should thrown Exception if the resource is not found
## What changes were proposed in this pull request?

Create temporary or permanent function it should throw AnalysisException if the resource is not found. Need to keep behavior consistent across permanent and temporary functions.

## How was this patch tested?

Added UT and also tested manually

**Before Fix**
If the UDF resource is not present then on creation of temporary function it throws AnalysisException where as for permanent function it does not throw. Permanent funtcion  throws AnalysisException only after select operation is performed.

**After Fix**

For temporary and permanent function check for the resource, if the UDF resource is not found then throw AnalysisException

![rt](https://user-images.githubusercontent.com/35216143/62781519-d1131580-bad5-11e9-9d58-69e65be86c03.png)

Closes #25399 from sandeep-katta/funcIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-28 14:35:33 +09:00
Kent Yao f0bf2eb006 [SPARK-30356][SQL] Codegen support for the function str_to_map
### What changes were proposed in this pull request?
`str_to_map ` has not implemented with codegen support, which prevents a query that contains this expression from being whole stage codegen-ed.
This PR removes `CodegenFallBack` from `StringToMap`, add the codegen support for it.

### Why are the changes needed?

improve codegen coverage and gain better perfomance

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

no

### How was this patch tested?

1. pass ComplexTypeSuite

2. manually review generated code

```java

-- !query 12
explain codegen select v, str_to_map(v) from values ('abc🅰️a,:'), (null), (''), ('1:2') t(v)
-- !query 12 schema
struct<plan:string>
-- !query 12 output
Found 1 WholeStageCodegen subtrees.
== Subtree 1 / 1 (maxMethodCodeSize:511; maxConstantPoolSize:188(0.29% used); numInnerClasses:0) ==
*Project [v#x, str_to_map(v#x, ,, :) AS str_to_map(v, ,, :)#x]
+- *LocalTableScan [v#x]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator localtablescan_input_0;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] project_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[] project_mutableStateArray_1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter[2];
/* 012 */
/* 013 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     localtablescan_input_0 = inputs[0];
/* 021 */     project_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 64);
/* 022 */     project_mutableStateArray_1[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(project_mutableStateArray_0[0], 8);
/* 023 */     project_mutableStateArray_1[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(project_mutableStateArray_0[0], 8);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   private void project_doConsume_0(InternalRow localtablescan_row_0, UTF8String project_expr_0_0, boolean project_exprIsNull_0_0) throws java.io.IOException {
/* 028 */     boolean project_isNull_1 = true;
/* 029 */     MapData project_value_1 = null;
/* 030 */
/* 031 */     if (!project_exprIsNull_0_0) {
/* 032 */       project_isNull_1 = false; // resultCode could change nullability.
/* 033 */
/* 034 */       int project_i_0 = 0;
/* 035 */       UTF8String[] project_kvs_0 = project_expr_0_0.split(((UTF8String) references[2] /* literal */), -1);
/* 036 */       while (project_i_0 < project_kvs_0.length) {
/* 037 */         UTF8String[] kv = project_kvs_0[project_i_0].split(((UTF8String) references[3] /* literal */), 2);
/* 038 */         UTF8String key = kv[0];
/* 039 */         UTF8String value = null;
/* 040 */         if (kv.length == 2) {
/* 041 */           value = kv[1];
/* 042 */         }
/* 043 */         ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[1] /* mapBuilder */).put(key, value);
/* 044 */         project_i_0++;
/* 045 */       }
/* 046 */       project_value_1 = ((org.apache.spark.sql.catalyst.util.ArrayBasedMapBuilder) references[1] /* mapBuilder */).build();
/* 047 */
/* 048 */     }
/* 049 */     project_mutableStateArray_0[0].reset();
/* 050 */
/* 051 */     project_mutableStateArray_0[0].zeroOutNullBytes();
/* 052 */
/* 053 */     if (project_exprIsNull_0_0) {
/* 054 */       project_mutableStateArray_0[0].setNullAt(0);
/* 055 */     } else {
/* 056 */       project_mutableStateArray_0[0].write(0, project_expr_0_0);
/* 057 */     }
/* 058 */
/* 059 */     if (project_isNull_1) {
/* 060 */       project_mutableStateArray_0[0].setNullAt(1);
/* 061 */     } else {
/* 062 */       final MapData project_tmpInput_0 = project_value_1;
/* 063 */       if (project_tmpInput_0 instanceof UnsafeMapData) {
/* 064 */         project_mutableStateArray_0[0].write(1, (UnsafeMapData) project_tmpInput_0);
/* 065 */       } else {
/* 066 */         // Remember the current cursor so that we can calculate how many bytes are
/* 067 */         // written later.
/* 068 */         final int project_previousCursor_0 = project_mutableStateArray_0[0].cursor();
/* 069 */
/* 070 */         // preserve 8 bytes to write the key array numBytes later.
/* 071 */         project_mutableStateArray_0[0].grow(8);
/* 072 */         project_mutableStateArray_0[0].increaseCursor(8);
/* 073 */
/* 074 */         // Remember the current cursor so that we can write numBytes of key array later.
/* 075 */         final int project_tmpCursor_0 = project_mutableStateArray_0[0].cursor();
/* 076 */
/* 077 */         final ArrayData project_tmpInput_1 = project_tmpInput_0.keyArray();
/* 078 */         if (project_tmpInput_1 instanceof UnsafeArrayData) {
/* 079 */           project_mutableStateArray_0[0].write((UnsafeArrayData) project_tmpInput_1);
/* 080 */         } else {
/* 081 */           final int project_numElements_0 = project_tmpInput_1.numElements();
/* 082 */           project_mutableStateArray_1[0].initialize(project_numElements_0);
/* 083 */
/* 084 */           for (int project_index_0 = 0; project_index_0 < project_numElements_0; project_index_0++) {
/* 085 */             project_mutableStateArray_1[0].write(project_index_0, project_tmpInput_1.getUTF8String(project_index_0));
/* 086 */           }
/* 087 */         }
/* 088 */
/* 089 */         // Write the numBytes of key array into the first 8 bytes.
/* 090 */         Platform.putLong(
/* 091 */           project_mutableStateArray_0[0].getBuffer(),
/* 092 */           project_tmpCursor_0 - 8,
/* 093 */           project_mutableStateArray_0[0].cursor() - project_tmpCursor_0);
/* 094 */
/* 095 */         final ArrayData project_tmpInput_2 = project_tmpInput_0.valueArray();
/* 096 */         if (project_tmpInput_2 instanceof UnsafeArrayData) {
/* 097 */           project_mutableStateArray_0[0].write((UnsafeArrayData) project_tmpInput_2);
/* 098 */         } else {
/* 099 */           final int project_numElements_1 = project_tmpInput_2.numElements();
/* 100 */           project_mutableStateArray_1[1].initialize(project_numElements_1);
/* 101 */
/* 102 */           for (int project_index_1 = 0; project_index_1 < project_numElements_1; project_index_1++) {
/* 103 */             if (project_tmpInput_2.isNullAt(project_index_1)) {
/* 104 */               project_mutableStateArray_1[1].setNull8Bytes(project_index_1);
/* 105 */             } else {
/* 106 */               project_mutableStateArray_1[1].write(project_index_1, project_tmpInput_2.getUTF8String(project_index_1));
/* 107 */             }
/* 108 */
/* 109 */           }
/* 110 */         }
/* 111 */
/* 112 */         project_mutableStateArray_0[0].setOffsetAndSizeFromPreviousCursor(1, project_previousCursor_0);
/* 113 */       }
/* 114 */     }
/* 115 */     append((project_mutableStateArray_0[0].getRow()));
/* 116 */
/* 117 */   }
/* 118 */
/* 119 */   protected void processNext() throws java.io.IOException {
/* 120 */     while ( localtablescan_input_0.hasNext()) {
/* 121 */       InternalRow localtablescan_row_0 = (InternalRow) localtablescan_input_0.next();
/* 122 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 123 */       boolean localtablescan_isNull_0 = localtablescan_row_0.isNullAt(0);
/* 124 */       UTF8String localtablescan_value_0 = localtablescan_isNull_0 ?
/* 125 */       null : (localtablescan_row_0.getUTF8String(0));
/* 126 */
/* 127 */       project_doConsume_0(localtablescan_row_0, localtablescan_value_0, localtablescan_isNull_0);
/* 128 */       if (shouldStop()) return;
/* 129 */     }
/* 130 */   }
/* 131 */
/* 132 */ }

-- !query 13
select v, str_to_map(v) from values ('abc🅰️a,:'), (null), (''), ('1:2') t(v)
-- !query 13 schema
struct<v:string,str_to_map(v, ,, :):map<string,string>>
-- !query 13 output
	{"":null}
1:2	{"1":"2"}
NULL	NULL
abc🅰️a,:	{"":"","abc":"a:a"}
```

Closes #27013 from yaooqinn/SPARK-30356.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 21:44:03 +08:00
Jungtaek Lim (HeartSaVioR) 7adf886792 [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
### What changes were proposed in this pull request?

This patch fixes the intermittent test failure on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite, getting ConnectException when querying to thrift server.
(https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/)

The relevant unit test log messages are following:

```
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is started.
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:HiveServer2 is started.
...
19/12/23 13:33:01.888 pool-1-thread-1 INFO ThriftServerWithSparkContextSuite: HiveThriftServer2 started successfully
...
19/12/23 13:33:01.909 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== TEST OUTPUT FOR o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

...
19/12/23 13:33:02.017 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Supplied authorities: localhost:15441
19/12/23 13:33:02.018 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Resolved authority: localhost:15441
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.121 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite WARN HiveConnection: Failed to connect to localhost:15441
19/12/23 13:33:02.124 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== FINISHED o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

19/12/23 13:33:02.143 Thread-35 INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 15441 with 5...500 worker threads
19/12/23 13:33:02.327 pool-1-thread-1 INFO HiveServer2: Shutting down HiveServer2
19/12/23 13:33:02.328 pool-1-thread-1 INFO ThriftCLIService: Thrift server has stopped
```
(Here the error is logged as `WARN HiveConnection: Failed to connect to localhost:15441` - the actual stack trace can be seen on Jenkins test summary.)

The reason of test failure: Thrift(Binary|Http)CLIService prepare and launch the service asynchronously (in new thread), which suites are not waiting for completion and just start running tests, ends up with race condition.

That can be easily reproduced, via adding artificial sleep in `ThriftBinaryCLIService.run()` here:
ba3f6330dd/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (L49)

(Note that `sleep` should be added before initializing server socket. E.g. Line 57)

This patch changes the test initialization logic to try executing simple query to wait until the service is available. The patch also refactors the code to apply the change both ThriftServerQueryTestSuite and ThriftServerWithSparkContextSuite easily.

### Why are the changes needed?

This patch fixes the intermittent failure observed here:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/

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

No

### How was this patch tested?

Artificially made the test fail consistently (by the approach described above), and confirmed the patch fixed the test.

Closes #27001 from HeartSaVioR/SPARK-30345.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 15:30:54 +08:00
lijunqing a2de20c0e6 [SPARK-30036][SQL] Fix: REPARTITION hint does not work with order by
### Why are the changes needed?
`EnsureRequirements` adds `ShuffleExchangeExec` (RangePartitioning) after Sort if `RoundRobinPartitioning` behinds it. This will cause 2 shuffles, and the number of partitions in the final stage is not the number specified by `RoundRobinPartitioning.

**Example SQL**
```
SELECT /*+ REPARTITION(5) */ * FROM test ORDER BY a
```

**BEFORE**
```
== Physical Plan ==
*(1) Sort [a#0 ASC NULLS FIRST], true, 0
+- Exchange rangepartitioning(a#0 ASC NULLS FIRST, 200), true, [id=#11]
   +- Exchange RoundRobinPartitioning(5), false, [id=#9]
      +- Scan hive default.test [a#0, b#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#0, b#1]
```

**AFTER**
```
== Physical Plan ==
*(1) Sort [a#0 ASC NULLS FIRST], true, 0
+- Exchange rangepartitioning(a#0 ASC NULLS FIRST, 5), true, [id=#11]
   +- Scan hive default.test [a#0, b#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#0, b#1]
```

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

### How was this patch tested?
Run suite Tests and add new test for this.

Closes #26946 from stczwd/RoundRobinPartitioning.

Lead-authored-by: lijunqing <lijunqing@baidu.com>
Co-authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 11:52:39 +08:00
Kengo Seki 59c014e120 [SPARK-30350][SQL] Fix ScalaReflection to use an empty array for getting its class object
### What changes were proposed in this pull request?

This PR fixes `ScalaReflection.arrayClassFor()` to use an empty array instead of a one-element array for getting its class object by reflection.

### Why are the changes needed?

Because it may reduce unnecessary memory allocation.

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

No

### How was this patch tested?

Ran the existing unit tests for sql/catalyst and confirmed that all of them succeeded.

Closes #27005 from sekikn/SPARK-30350.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-12-26 22:54:29 +09:00
gengjiaan d59e7195f6 [SPARK-27986][SQL] Support ANSI SQL filter clause for aggregate expression
### What changes were proposed in this pull request?
The filter predicate for aggregate expression is an `ANSI SQL`.
```
<aggregate function> ::=
COUNT <left paren> <asterisk> <right paren> [ <filter clause> ]
| <general set function> [ <filter clause> ]
| <binary set function> [ <filter clause> ]
| <ordered set function> [ <filter clause> ]
| <array aggregate function> [ <filter clause> ]
| <row pattern count function> [ <filter clause> ]
```
There are some mainstream database support this syntax.
**PostgreSQL:**
https://www.postgresql.org/docs/current/sql-expressions.html#SYNTAX-AGGREGATES
For example:
```
SELECT
  year,
  count(*) FILTER (WHERE gdp_per_capita >= 40000)
FROM
  countries
GROUP BY
  year
```
```
SELECT
  year,
  code,
  gdp_per_capita,
  count(*)
    FILTER (WHERE gdp_per_capita >= 40000)
    OVER   (PARTITION BY year)
FROM
  countries
```
**jOOQ:**
https://blog.jooq.org/2014/12/30/the-awesome-postgresql-9-4-sql2003-filter-clause-for-aggregate-functions/

**Notice:**
1.This PR only supports FILTER predicate without codegen. maropu will create another PR is related to SPARK-30027 to support codegen.
2.This PR only supports FILTER predicate without DISTINCT. I will create another PR is related to SPARK-30276 to support this.
3.This PR only supports FILTER predicate that can't reference the outer query. I created ticket SPARK-30219 to support it.
4.This PR only supports FILTER predicate that can't use IN/EXISTS predicate sub-queries. I created ticket SPARK-30220 to support it.
5.Spark SQL cannot supports a SQL with nested aggregate. I created ticket SPARK-30182 to support it.

There are some show of the PR on my production environment.
```
spark-sql> desc gja_test_partition;
key     string  NULL
value   string  NULL
other   string  NULL
col2    int     NULL
# Partition Information
# col_name      data_type       comment
col2    int     NULL
Time taken: 0.79 s
```
```
spark-sql> select * from gja_test_partition;
a       A       ao      1
b       B       bo      1
c       C       co      1
d       D       do      1
e       E       eo      2
g       G       go      2
h       H       ho      2
j       J       jo      2
f       F       fo      3
k       K       ko      3
l       L       lo      4
i       I       io      4
Time taken: 1.75 s
```
```
spark-sql> select count(key), sum(col2) from gja_test_partition;
12      26
Time taken: 1.848 s
```
```
spark-sql> select count(key) filter (where col2 > 1) from gja_test_partition;
8
Time taken: 2.926 s
```
```
spark-sql> select sum(col2) filter (where col2 > 2) from gja_test_partition;
14
Time taken: 2.087 s
```
```
spark-sql> select count(key) filter (where col2 > 1), sum(col2) filter (where col2 > 2) from gja_test_partition;
8       14
Time taken: 2.847 s
```
```
spark-sql> select count(key), count(key) filter (where col2 > 1), sum(col2), sum(col2) filter (where col2 > 2) from gja_test_partition;
12      8       26      14
Time taken: 1.787 s
```
```
spark-sql> desc student;
id      int     NULL
name    string  NULL
sex     string  NULL
class_id        int     NULL
Time taken: 0.206 s
```
```
spark-sql> select * from student;
1       张三    man     1
2       李四    man     1
3       王五    man     2
4       赵六    man     2
5       钱小花  woman   1
6       赵九红  woman   2
7       郭丽丽  woman   2
Time taken: 0.786 s
```
```
spark-sql> select class_id, count(id), sum(id) from student group by class_id;
1       3       8
2       4       20
Time taken: 18.783 s
```
```
spark-sql> select class_id, count(id) filter (where sex = 'man'), sum(id) filter (where sex = 'woman') from student group by class_id;
1       2       5
2       2       13
Time taken: 3.887 s
```

### Why are the changes needed?
Add new SQL feature.

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

### How was this patch tested?
Exists UT and new UT.

Closes #26656 from beliefer/support-aggregate-clause.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-26 17:41:50 +08:00