Commit graph

11704 commits

Author SHA1 Message Date
shahid cd2ef9cb43 [SPARK-35567][SQL] Fix: Explain cost is not showing statistics for all the nodes
### What changes were proposed in this pull request?
Explain cost command in spark currently doesn't show statistics for all the nodes. It misses some nodes in almost all the TPCDS queries.
In this PR, we are collecting all the plan nodes including the subqueries and computing  the statistics for each node, if it doesn't exists in stats cache,

### Why are the changes needed?
**Before Fix**
For eg: Query1,  Project node doesn't have statistics
![image](https://user-images.githubusercontent.com/23054875/120123442-868feb00-c1cc-11eb-9af9-3a87bf2117d2.png)

Query15, Aggregate node doesn't have statistics

![image](https://user-images.githubusercontent.com/23054875/120123296-a4108500-c1cb-11eb-89df-7fddd651572e.png)

**After Fix:**
Query1:
![image](https://user-images.githubusercontent.com/23054875/120123559-1df53e00-c1cd-11eb-938a-53704f5240e6.png)
Query 15:
![image](https://user-images.githubusercontent.com/23054875/120123665-bb507200-c1cd-11eb-8ea2-84c732215bac.png)
### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual testing

Closes #32704 from shahidki31/shahid/fixshowstats.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-01 00:55:29 +08:00
Tengfei Huang 1603775934 [SPARK-35411][SQL][FOLLOWUP] Handle Currying Product while serializing TreeNode to JSON
### What changes were proposed in this pull request?
Handle Currying Product while serializing TreeNode to JSON. While processing [Product](https://github.com/apache/spark/blob/v3.1.2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala#L820), we may get an assert error for cases like Currying Product because of the mismatch of sizes between field name and field values.
Fallback to use reflection to get all the values for constructor parameters when we  meet such cases.

### Why are the changes needed?
Avoid throwing error while serializing TreeNode to JSON, try to output as much information as possible.

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

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

Closes #32713 from ivoson/SPARK-35411-followup.

Authored-by: Tengfei Huang <tengfei.h@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-31 22:15:26 +08:00
Yuming Wang 6cd6c438f2 [SPARK-34808][SQL] Removes outer join if it only has DISTINCT on streamed side
### What changes were proposed in this pull request?

This pr add new rule to removes outer join if it only has distinct on streamed side. For example:
```scala
spark.range(200L).selectExpr("id AS a").createTempView("t1")
spark.range(300L).selectExpr("id AS b").createTempView("t2")
spark.sql("SELECT DISTINCT a FROM t1 LEFT JOIN t2 ON a = b").explain(true)
```

Before this pr:
```
== Optimized Logical Plan ==
Aggregate [a#2L], [a#2L]
+- Project [a#2L]
   +- Join LeftOuter, (a#2L = b#6L)
      :- Project [id#0L AS a#2L]
      :  +- Range (0, 200, step=1, splits=Some(2))
      +- Project [id#4L AS b#6L]
         +- Range (0, 300, step=1, splits=Some(2))
```

After this pr:
```
== Optimized Logical Plan ==
Aggregate [a#2L], [a#2L]
+- Project [id#0L AS a#2L]
   +- Range (0, 200, step=1, splits=Some(2))
```

### Why are the changes needed?

Improve query performance. [DB2](https://www.ibm.com/docs/en/db2-for-zos/11?topic=manipulation-how-db2-simplifies-join-operations) support this feature:
![image](https://user-images.githubusercontent.com/5399861/119594277-0d7c4680-be0e-11eb-8bd4-366d8c4639f0.png)

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

No.

### How was this patch tested?

Unit test.

Closes #31908 from wangyum/SPARK-34808.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-05-31 18:14:15 +08:00
Liang-Chi Hsieh 73ba4492b1 [SPARK-35566][SS] Fix StateStoreRestoreExec output rows
### What changes were proposed in this pull request?

This is a minor change to update how `StateStoreRestoreExec` computes its number of output rows. Previously we only count input rows, but the optionally restored rows are not counted in.

### Why are the changes needed?

Currently the number of output rows of `StateStoreRestoreExec` only counts the each input row. But it actually outputs input rows + optional restored rows. We should provide correct number of output rows.

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

No

### How was this patch tested?

Existing tests.

Closes #32703 from viirya/fix-outputrows.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-31 16:45:56 +09:00
allisonwang-db 806da9d6fa [SPARK-35545][SQL] Split SubqueryExpression's children field into outer attributes and join conditions
### What changes were proposed in this pull request?
This PR refactors `SubqueryExpression` class. It removes the children field from SubqueryExpression's constructor and adds `outerAttrs` and `joinCond`.

### Why are the changes needed?
Currently, the children field of a subquery expression is used to store both collected outer references in the subquery plan and join conditions after correlated predicates are pulled up.

For example:
`SELECT (SELECT max(c1) FROM t1 WHERE t1.c1 = t2.c1) FROM t2`

During the analysis phase, outer references in the subquery are stored in the children field: `scalar-subquery [t2.c1]`, but after the optimizer rule `PullupCorrelatedPredicates`, the children field will be used to store the join conditions, which contain both the inner and the outer references: `scalar-subquery [t1.c1 = t2.c1]`. This is why the references of SubqueryExpression excludes the inner plan's output:
29ed1a2de4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala (L68-L69)

This can be confusing and error-prone. The references for a subquery expression should always be defined as outer attribute references.

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

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

Closes #32687 from allisonwang-db/refactor-subquery-expr.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-31 04:57:24 +00:00
yangjie01 09d039da56 [SPARK-35526][CORE][SQL][ML][MLLIB] Re-Cleanup procedure syntax is deprecated compilation warning in Scala 2.13
### What changes were proposed in this pull request?
After SPARK-29291 and SPARK-33352, there are still some compilation warnings about `procedure syntax is deprecated` as follows:

```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:723: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `registerMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:748: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `unregisterMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala:223: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testSimpleSpillingForAllCodecs`'s return type
[WARNING] [Warn] /spark/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala:53: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `runBLASBenchmark`'s return type
[WARNING] [Warn] /spark/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala:110: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `assertEmptyRootPath`'s return type
[WARNING] [Warn] /spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:602: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `executeCTASWithNonEmptyLocation`'s return type
```

So the main change of this pr is cleanup these compilation warnings.

### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #32669 from LuciferYang/re-clean-procedure-syntax.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-30 16:49:47 -07:00
Yingyi Bu 5c8a141d03 [SPARK-35538][SQL] Migrate transformAllExpressions call sites to use transformAllExpressionsWithPruning
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- EXCHANGE
- IN_SUBQUERY_EXEC
- UPDATE_FIELDS

Migrated `transformAllExpressions` call sites to use `transformAllExpressionsWithPruning`

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

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

No.

### How was this patch tested?

Existing tests.
Perf diff:
Rule name | Total Time (baseline) | Total Time (experiment) | experiment/baseline
OptimizeUpdateFields | 54646396 | 27444424 | 0.5
ReplaceUpdateFieldsExpression  | 24694303 | 2087517 | 0.08

Closes #32643 from sigmod/all_expressions.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2021-05-28 15:36:25 -07:00
Wenchen Fan 678592a612 [SPARK-35559][TEST] Speed up one test in AdaptiveQueryExecSuite
### What changes were proposed in this pull request?

I just noticed that `AdaptiveQueryExecSuite.SPARK-34091: Batch shuffle fetch in AQE partition coalescing` takes more than 10 minutes to finish, which is unacceptable.

This PR sets the shuffle partitions to 10 in that test, so that the test can finish with 5 seconds.

### Why are the changes needed?

speed up the test

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

no

### How was this patch tested?

N/A

Closes #32695 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-28 12:39:34 -07:00
Kousuke Saruta b763db3efd [SPARK-35194][SQL][FOLLOWUP] Recover build error with Scala 2.13 on GA
### What changes were proposed in this pull request?

This PR fixes a build error with Scala 2.13 on GA.
#32301 seems to bring this error.

### Why are the changes needed?

To recover CI.

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

No.

### How was this patch tested?

GA

Closes #32696 from sarutak/followup-SPARK-35194.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-29 00:11:16 +09:00
Karen Feng e8631660ec [SPARK-35194][SQL] Refactor nested column aliasing for readability
### What changes were proposed in this pull request?

Refactors `NestedColumnAliasing` and `GeneratorNestedColumnAliasing` for readability.

### Why are the changes needed?

Improves readability for future maintenance.

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

No.

### How was this patch tested?

Existing tests.

Closes #32301 from karenfeng/refactor-nested-column-aliasing.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-28 13:18:44 +00:00
ulysses-you 3b94aad5e7 [SPARK-35552][SQL] Make query stage materialized more readable
### What changes were proposed in this pull request?

Add a new method `isMaterialized` in `QueryStageExec`.

### Why are the changes needed?

Currently, we use `resultOption().get.isDefined` to check if a query stage has materialized. The code is not readable at a glance. It's better to use a new method like `isMaterialized` to define it.

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

No.

### How was this patch tested?

Pass CI.

Closes #32689 from ulysses-you/SPARK-35552.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-05-28 20:42:11 +08:00
Wenchen Fan 29ed1a2de4 [SPARK-35541][SQL] Simplify OptimizeSkewedJoin
### What changes were proposed in this pull request?

Various small code simplification/cleanup for OptimizeSkewedJoin

### Why are the changes needed?

code refactor

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

no

### How was this patch tested?

existing tests

Closes #32685 from cloud-fan/skew-join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-27 09:17:28 -07:00
Yuanjian Li f98a063a4b [SPARK-35172][SS] The implementation of RocksDBCheckpointMetadata
### What changes were proposed in this pull request?
Initial implementation of RocksDBCheckpointMetadata. It persists the metadata for RocksDBFileManager.

### Why are the changes needed?
The RocksDBCheckpointMetadata persists the metadata for each committed batch in JSON format. The object contains all RocksDB file names and the number of total keys.
The metadata binds closely with the directory structure of RocksDBFileManager, as described in the design doc - [Directory Structure and Format for Files stored in DFS](https://docs.google.com/document/d/10wVGaUorgPt4iVe4phunAcjU924fa3-_Kf29-2nxH6Y/edit#heading=h.zgvw85ijoz2).

### Does this PR introduce _any_ user-facing change?
No. Internal implementation only.

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

Closes #32272 from xuanyuanking/SPARK-35172.

Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-27 22:56:50 +09:00
dgd-contributor 52a1f8c000 [SPARK-33428][SQL] Match the behavior of conv function to MySQL's
### What changes were proposed in this pull request?
Spark conv function is from MySQL and it's better to follow the MySQL behavior. MySQL returns the max unsigned long if the input string is too big, and Spark should follow it.

However, seems Spark has different behavior in two cases:

MySQL allows leading spaces but Spark does not.
If the input string is way too long, Spark fails with ArrayIndexOutOfBoundException

This patch now help conv follow behavior in those two cases
conv allows leading spaces
conv will return the max unsigned long when the input string is way too long

### Why are the changes needed?
fixing it to match the behavior of conv function to the (almost) only one reference of another DBMS, MySQL

### Does this PR introduce _any_ user-facing change?
Yes, as pointed out above

### How was this patch tested?
Add test

Closes #32684 from dgd-contributor/SPARK-33428.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-27 12:12:39 +00:00
Gengliang Wang 5bcd1c29f0 [SPARK-35535][SQL] New data source V2 API: LocalScan
### What changes were proposed in this pull request?

Add a new data source V2 API: `LocalScan`. It is a special Scan that will happen on Driver locally instead of Executors.

### Why are the changes needed?

The new API improves the flexibility of the DSV2 API. It allows developers to implement connectors for data sources of small data sizes.
For example, we can build a data source for Spark History applications from Spark History Server RESTFUL API. The result set is small and fetching all the results from the Spark driver is good enough. Making it a data source allows us to operate SQL queries with filters or table joins.

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

No.

### How was this patch tested?

Unit test

Closes #32678 from gengliangwang/LocalScan.

Lead-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-27 19:31:56 +09:00
gengjiaan 3e190807bc [SPARK-35057][SQL] Group exception messages in hive/thriftserver
### What changes were proposed in this pull request?
This PR group exception messages in `sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32646 from beliefer/SPARK-35057.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-27 07:31:14 +00:00
Cheng Su 5cc17ba0c7 [SPARK-35351][SQL][FOLLOWUP] Avoid using loaded variable for LEFT ANTI SMJ code-gen
### What changes were proposed in this pull request?

This is a followup from https://github.com/apache/spark/pull/32547#discussion_r639916474, where for LEFT ANTI join, we do not need to depend on `loaded` variable, as in `codegenAnti` we only load `streamedAfter` no more than once (i.e. assign column values from streamed row which are not used in join condition).

### Why are the changes needed?

Avoid unnecessary processing in code-gen (though it's just `boolean $loaded = false;`, and `if (!$loaded) { $loaded = true; }`).

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

No.

### How was this patch tested?

Existing unite tests in `ExistenceJoinSuite`.

Closes #32681 from c21/join-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-27 04:59:54 +00:00
ulysses-you dc7b5a99f0 [SPARK-35282][SQL] Support AQE side shuffled hash join formula using rule
### What changes were proposed in this pull request?

The main code change is:
* Change rule `DemoteBroadcastHashJoin` to `DynamicJoinSelection` and add shuffle hash join selection code.
* Specify a join strategy hint `SHUFFLE_HASH` if AQE think a join can be converted to SHJ.
* Skip `preferSortMerge` config check in AQE side if a join can be converted to SHJ.

### Why are the changes needed?

Use AQE runtime statistics to decide if we can use shuffled hash join instead of sort merge join. Currently, the formula of shuffled hash join selection dose not work due to the dymanic shuffle partition number.

Add a new config spark.sql.adaptive.shuffledHashJoinLocalMapThreshold to decide if join can be converted to shuffled hash join safely.

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

Yes, add a new config.

### How was this patch tested?

Add test.

Closes #32550 from ulysses-you/SPARK-35282-2.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 14:16:04 +00:00
Cheng Su dd677770d8 [SPARK-35529][SQL] Add fallback metrics for hash aggregate
### What changes were proposed in this pull request?

Add the metrics to record how many tasks fallback to sort-based aggregation for hash aggregation. This will help developers and users to debug and optimize query. Object hash aggregation has similar metrics already.

### Why are the changes needed?

Help developers and users to debug and optimize query with hash aggregation.

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

Yes, the added metrics will show up in Spark web UI.
Example:
<img width="604" alt="Screen Shot 2021-05-26 at 12 17 08 AM" src="https://user-images.githubusercontent.com/4629931/119618437-bf3c5880-bdb7-11eb-89bb-5b88db78639f.png">

### How was this patch tested?

Changed unit test in `SQLMetricsSuite.scala`.

Closes #32671 from c21/agg-metrics.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 11:28:12 +00:00
Kousuke Saruta 50fefc6447 [SPARK-35527][SQL][TESTS] Fix HiveExternalCatalogVersionsSuite to pass with Java 11
### What changes were proposed in this pull request?

This PR fixes `HiveExternalCatalogVersionsSuite`.
With this change, only <major>.<minor> version is set to `spark.sql.hive.metastore.version`.

### Why are the changes needed?

I'm personally checking whether all the tests pass with Java 11 for the current `master` and I found `HiveExternalCatalogVersionsSuite` fails.
The reason is that Spark 3.0.2 and 3.1.1 doesn't accept `2.3.8` as a hive metastore version.

`HiveExternalCatalogVersionsSuite` downloads Spark releases from https://dist.apache.org/repos/dist/release/spark/ and run test for each release. The Spark releases are `3.0.2` and `3.1.1` for the current `master` for now.
e47e615c0e/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala (L239-L259)

With Java 11, the suite run with a hive metastore version which corresponds to the builtin Hive version and it's `2.3.8` for the current `master`.
20750a3f9e/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala (L62-L66)

But `branch-3.0` and `branch-3.1` doesn't accept `2.3.8`, the suite with Java 11 fails.

Another solution would be backporting SPARK-34271 (#31371) but after [a discussion](https://github.com/apache/spark/pull/32668#issuecomment-848435170), we prefer to fix the test,

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

No.

### How was this patch tested?

Existing tests with CI.

Closes #32670 from sarutak/fix-version-suite-for-java11.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-26 17:20:51 +09:00
itholic 79a6b0cc8a [SPARK-35509][DOCS] Move text data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move text data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for text data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "Text Files" page
<img width="823" alt="Screen Shot 2021-05-26 at 3 20 11 PM" src="https://user-images.githubusercontent.com/44108233/119611669-f5202200-be35-11eb-9307-45846949d300.png">

- Python
<img width="791" alt="Screen Shot 2021-05-25 at 5 04 26 PM" src="https://user-images.githubusercontent.com/44108233/119462469-b9c11d00-bd7b-11eb-8f19-2ba7b9ceb318.png">

- Scala
<img width="683" alt="Screen Shot 2021-05-25 at 5 05 10 PM" src="https://user-images.githubusercontent.com/44108233/119462483-bd54a400-bd7b-11eb-8177-74e4d7035e63.png">

- Java
<img width="665" alt="Screen Shot 2021-05-25 at 5 05 36 PM" src="https://user-images.githubusercontent.com/44108233/119462501-bfb6fe00-bd7b-11eb-8161-12c58fabe7e2.png">

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32660 from itholic/SPARK-35509.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-26 17:12:49 +09:00
Vinod KC e3c6907c99 [SPARK-35490][BUILD] Update json4s to 3.7.0-M11
### What changes were proposed in this pull request?
This PR aims to upgrade json4s from   3.7.0-M5  to 3.7.0-M11

Note: json4s version greater than 3.7.0-M11 is not binary compatible with Spark third party jars

### Why are the changes needed?
Multiple defect fixes and improvements  like

https://github.com/json4s/json4s/issues/750
https://github.com/json4s/json4s/issues/554
https://github.com/json4s/json4s/issues/715

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

### How was this patch tested?
Ran with the existing UTs

Closes #32636 from vinodkc/br_build_upgrade_json4s.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-26 11:10:14 +03:00
Linhong Liu af1dba7ca5 [SPARK-35440][SQL] Add function type to ExpressionInfo for UDF
### What changes were proposed in this pull request?
Add the function type, such as "scala_udf", "python_udf", "java_udf", "hive", "built-in" to the `ExpressionInfo` for UDF.

### Why are the changes needed?
Make the `ExpressionInfo` of UDF more meaningful

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

### How was this patch tested?
existing and newly added UT

Closes #32587 from linhongliu-db/udf-language.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 04:40:53 +00:00
Hyukjin Kwon 20750a3f9e [SPARK-32194][PYTHON] Use proper exception classes instead of plain Exception
### What changes were proposed in this pull request?

This PR proposes to use a proper built-in exceptions instead of the plain `Exception` in Python.

While I am here, I fixed another minor issue at `DataFrams.schema` together:

```diff
- except AttributeError as e:
-     raise Exception(
-         "Unable to parse datatype from schema. %s" % e)
+ except Exception as e:
+     raise ValueError(
+         "Unable to parse datatype from schema. %s" % e) from e
```

Now it catches all exceptions during schema parsing, chains the exception with `ValueError`. Previously it only caught `AttributeError` that does not catch all cases.

### Why are the changes needed?

For users to expect the proper exceptions.

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

Yeah, the exception classes became different but should be compatible because previous exception was plain `Exception` which other exceptions inherit.

### How was this patch tested?

Existing unittests should cover,

Closes #31238

Closes #32650 from HyukjinKwon/SPARK-32194.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-26 11:54:40 +09:00
Wenchen Fan 859a53424a [SPARK-35447][SQL] Optimize skew join before coalescing shuffle partitions
### What changes were proposed in this pull request?

This PR improves the interaction between partition coalescing and skew handling by moving the skew join rule ahead of the partition coalescing rule and making corresponding changes to the two rules:
1. Simplify `OptimizeSkewedJoin` as it doesn't need to handle `CustomShuffleReaderExec` anymore.
2. Update `CoalesceShufflePartitions` to support coalescing non-skewed partitions.

### Why are the changes needed?

It's a bit hard to reason about skew join if partitions have been coalesced. A skewed partition needs to be much larger than other partitions and we need to look at the raw sizes before coalescing.

It also makes `OptimizeSkewedJoin` more robust, as we don't need to worry about a skewed partition being coalesced with a small partition and breaks skew join handling.

It also helps with https://github.com/apache/spark/pull/31653 , which needs to move `OptimizeSkewedJoin` to an earlier phase and run before `CoalesceShufflePartitions`.

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

No

### How was this patch tested?

new UT and existing tests

Closes #32594 from cloud-fan/shuffle.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 13:12:45 +00:00
ulysses-you 631077db08 [SPARK-35455][SQL] Unify empty relation optimization between normal and AQE optimizer
### What changes were proposed in this pull request?

* remove `EliminateUnnecessaryJoin`, using `AQEPropagateEmptyRelation` instead.
* eliminate join, aggregate, limit, repartition, sort, generate which is beneficial.

### Why are the changes needed?

Make `EliminateUnnecessaryJoin` available with more case.

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

No.

### How was this patch tested?

Add test.

Closes #32602 from ulysses-you/SPARK-35455.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 08:59:59 +00:00
tanel.kiis@gmail.com 548e37b00b [SPARK-33122][SQL][FOLLOWUP] Extend RemoveRedundantAggregates optimizer rule to apply to more cases
### What changes were proposed in this pull request?

Addressed the dongjoon-hyun comments on the previous PR #30018.
Extended the `RemoveRedundantAggregates` rule to remove redundant aggregations in even more queries. For example in
 ```
dataset
    .dropDuplicates()
    .groupBy('a)
    .agg(max('b))
```
the `dropDuplicates` is not needed, because the result on `max` does not depend on duplicate values.

### Why are the changes needed?

Improve performance.

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

No

### How was this patch tested?

UT

Closes #31914 from tanelk/SPARK-33122_redundant_aggs_followup.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-25 10:04:37 +09:00
Kousuke Saruta d4fb98354a [SPARK-35287][SQL] Allow RemoveRedundantProjects to preserve ProjectExec which generates UnsafeRow for DataSourceV2ScanRelation
### What changes were proposed in this pull request?

This PR fixes an issue that `RemoveRedundantProjects` removes `ProjectExec` which is for generating `UnsafeRow`.
In `DataSourceV2Strategy`, `ProjectExec` will be inserted to ensure internal rows are `UnsafeRow`.

```
  private def withProjectAndFilter(
      project: Seq[NamedExpression],
      filters: Seq[Expression],
      scan: LeafExecNode,
      needsUnsafeConversion: Boolean): SparkPlan = {
    val filterCondition = filters.reduceLeftOption(And)
    val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)

    if (withFilter.output != project || needsUnsafeConversion) {
      ProjectExec(project, withFilter)
    } else {
      withFilter
    }
  }
...
    case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) =>
      // projection and filters were already pushed down in the optimizer.
      // this uses PhysicalOperation to get the projection and ensure that if the batch scan does
      // not support columnar, a projection is added to convert the rows to UnsafeRow.
      val batchExec = BatchScanExec(relation.output, relation.scan)
      withProjectAndFilter(project, filters, batchExec, !batchExec.supportsColumnar) :: Nil
```
So, the hierarchy of the partial tree should be like `ProjectExec(FilterExec(BatchScan))`.
But `RemoveRedundantProjects` doesn't consider this type of hierarchy, leading `ClassCastException`.

A concreate example to reproduce this issue is reported:
```
import scala.collection.JavaConverters._

import org.apache.iceberg.{PartitionSpec, TableProperties}
import org.apache.iceberg.hadoop.HadoopTables
import org.apache.iceberg.spark.SparkSchemaUtil
import org.apache.spark.sql.{DataFrame, QueryTest, SparkSession}
import org.apache.spark.sql.internal.SQLConf

class RemoveRedundantProjectsTest extends QueryTest {
  override val spark: SparkSession = SparkSession
    .builder()
    .master("local[4]")
    .config("spark.driver.bindAddress", "127.0.0.1")
    .appName(suiteName)
    .getOrCreate()
  test("RemoveRedundantProjects removes non-redundant projects") {
    withSQLConf(
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
      SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false",
      SQLConf.REMOVE_REDUNDANT_PROJECTS_ENABLED.key -> "true") {
      withTempDir { dir =>
        val path = dir.getCanonicalPath
        val data = spark.range(3).toDF
        val table = new HadoopTables().create(
          SparkSchemaUtil.convert(data.schema),
          PartitionSpec.unpartitioned(),
          Map(TableProperties.WRITE_NEW_DATA_LOCATION -> path).asJava,
          path)
        data.write.format("iceberg").mode("overwrite").save(path)
        table.refresh()

        val df = spark.read.format("iceberg").load(path)
        val dfX = df.as("x")
        val dfY = df.as("y")
        val join = dfX.filter(dfX("id") > 0).join(dfY, "id")
        join.explain("extended")
        assert(join.count() == 2)
      }
    }
  }
}
```
```
[info] - RemoveRedundantProjects removes non-redundant projects *** FAILED ***
[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 4) (xeroxms100.northamerica.corp.microsoft.com executor driver): java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
[info]  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:226)
[info]  at org.apache.spark.sql.execution.SortExec.$anonfun$doExecute$1(SortExec.scala:119)
```

### Why are the changes needed?

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

No.

### How was this patch tested?

New test.

Closes #32606 from sarutak/fix-project-removal-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 00:26:10 +08:00
Chao Sun c709efc1e7 [SPARK-34981][SQL][FOLLOWUP] Use SpecificInternalRow in ApplyFunctionExpression
### What changes were proposed in this pull request?

Use `SpecificInternalRow` instead of `GenericInternalRow` to avoid boxing / unboxing cost.

### Why are the changes needed?

Since it doesn't know the input row schema, `GenericInternalRow` potentially need to apply boxing for input arguments. It's better to use `SpecificInternalRow` instead since we know input data types.

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

No.

### How was this patch tested?

Existing tests.

Closes #32647 from sunchao/specific-input-row.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-24 17:25:24 +09:00
Adam Binford 6c0c617bd0 [SPARK-35449][SQL] Only extract common expressions from CaseWhen values if elseValue is set
### What changes were proposed in this pull request?

This PR fixes a bug with subexpression elimination for CaseWhen statements. https://github.com/apache/spark/pull/30245 added support for creating subexpressions that are present in all branches of conditional statements. However, for a statement to be in "all branches" of a CaseWhen statement, it must also be in the elseValue.

### Why are the changes needed?

Fix a bug where a subexpression can be created and run for branches of a conditional that don't pass. This can cause issues especially with a UDF in a branch that gets executed assuming the condition is true.

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

Yes, fixes a potential bug where a UDF could be eagerly executed even though it might expect to have already passed some form of validation. For example:
```
val col = when($"id" < 0, myUdf($"id"))
spark.range(1).select(when(col > 0, col)).show()
```

`myUdf($"id")` is considered a subexpression and eagerly evaluated, because it is pulled out as a common expression from both executions of the when clause, but if `id >= 0` it should never actually be run.

### How was this patch tested?

Updated existing test with new case.

Closes #32595 from Kimahriman/bug-case-subexpr-elimination.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-24 00:27:41 -07:00
Liang-Chi Hsieh 9e1b204bcc [SPARK-35410][SQL] SubExpr elimination should not include redundant children exprs in conditional expression
### What changes were proposed in this pull request?

This patch fixes a bug when dealing with common expressions in conditional expressions such as `CaseWhen` during subexpression elimination.

For example, previously we find common expressions among conditions of `CaseWhen`, but children expressions are also counted into. We should not count these children expressions as common expressions.

### Why are the changes needed?

If the redundant children expressions are counted as common expressions too, they will be redundantly evaluated and miss the subexpression elimination opportunity.

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

No

### How was this patch tested?

Added tests.

Closes #32559 from viirya/SPARK-35410.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-23 08:24:44 -07:00
Hyukjin Kwon 1d9f09decb [SPARK-35480][SQL] Make percentile_approx work with pivot
### What changes were proposed in this pull request?

This PR proposes to avoid wrapping if-else to the constant literals for `percentage` and `accuracy` in `percentile_approx`. They are expected to be literals (or foldable expressions).

Pivot works by two phrase aggregations, and it works with manipulating the input to `null` for non-matched values (pivot column and value).

Note that pivot supports an optimized version without such logic with changing input to `null` for some types (non-nested types basically). So the issue fixed by this PR is only for complex types.

```scala
val df = Seq(
  ("a", -1.0), ("a", 5.5), ("a", 2.5), ("b", 3.0), ("b", 5.2)).toDF("type", "value")
  .groupBy().pivot("type", Seq("a", "b")).agg(
    percentile_approx(col("value"), array(lit(0.5)), lit(10000)))
df.show()
```

**Before:**

```
org.apache.spark.sql.AnalysisException: cannot resolve 'percentile_approx((IF((type <=> CAST('a' AS STRING)), value, CAST(NULL AS DOUBLE))), (IF((type <=> CAST('a' AS STRING)), array(0.5D), NULL)), (IF((type <=> CAST('a' AS STRING)), 10000, CAST(NULL AS INT))))' due to data type mismatch: The accuracy or percentage provided must be a constant literal;
'Aggregate [percentile_approx(if ((type#7 <=> cast(a as string))) value#8 else cast(null as double), if ((type#7 <=> cast(a as string))) array(0.5) else cast(null as array<double>), if ((type#7 <=> cast(a as string))) 10000 else cast(null as int), 0, 0) AS a#16, percentile_approx(if ((type#7 <=> cast(b as string))) value#8 else cast(null as double), if ((type#7 <=> cast(b as string))) array(0.5) else cast(null as array<double>), if ((type#7 <=> cast(b as string))) 10000 else cast(null as int), 0, 0) AS b#18]
+- Project [_1#2 AS type#7, _2#3 AS value#8]
   +- LocalRelation [_1#2, _2#3]
```

**After:**

```
+-----+-----+
|    a|    b|
+-----+-----+
|[2.5]|[3.0]|
+-----+-----+
```

### Why are the changes needed?

To make percentile_approx work with pivot as expected

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

Yes. It threw an exception but now it returns a correct result as shown above.

### How was this patch tested?

Manually tested and unit test was added.

Closes #32619 from HyukjinKwon/SPARK-35480.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-23 07:35:43 +09:00
Wenchen Fan b624b7e93f [SPARK-28551][SQL][FOLLOWUP] Use the corrected hadoop conf
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32411, to fix a mistake and use `sparkSession.sessionState.newHadoopConf` which includes SQL configs instead of `sparkSession.sparkContext.hadoopConfiguration` .

### Why are the changes needed?

fix mistake

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

no

### How was this patch tested?

existing tests

Closes #32618 from cloud-fan/follow1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-22 10:33:57 +08:00
Liang-Chi Hsieh 066944c1bd [SPARK-35439][SQL] Children subexpr should come first than parent subexpr
### What changes were proposed in this pull request?

This patch sorts equivalent expressions based on their child-parent relation.

### Why are the changes needed?

`EquivalentExpressions` maintains a map of equivalent expressions. It is `HashMap` now so the insertion order is not guaranteed to be preserved later. Subexpression elimination relies on retrieving subexpressions from the map. If there is child-parent relationships among the subexpressions, we want the child expressions come first than parent expressions, so we can replace child expressions in parent expressions with subexpression evaluation.

For example, we have two different expressions `Add(Literal(1), Literal(2))` and `Add(Literal(3), add)`.

Case 1: child subexpr comes first.
```scala
addExprTree(add)
addExprTree(Add(Literal(3), add))
addExprTree(Add(Literal(3), add))
```

Case 2: parent subexpr comes first. For this case, we need to sort equivalent expressions.
```
addExprTree(Add(Literal(3), add))  => We add `Add(Literal(3), add)` into the map first, then add `add` into the map
addExprTree(add)
addExprTree(Add(Literal(3), add))
```

As we are going to sort equivalent expressions at all, we don't need `LinkedHashMap` but just do sorting.

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

No

### How was this patch tested?

Added tests.

Closes #32586 from viirya/use-listhashmap.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-21 10:49:35 -07:00
Dongjoon Hyun cc05daa884 [SPARK-34558][SQL][TESTS][FOLLOWUP] Fix a test to use a unknown filesystem
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32622 to fix a test case.

### Why are the changes needed?

Fix a wrong test case name and fix the test case to cause the expected error correctly.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #32623 from dongjoon-hyun/SPARK-34558.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-21 10:13:49 -07:00
Wenchen Fan 7274e3a4d2 [SPARK-34558][SQL][FOLLOWUP] Do not fail Spark startup with a broken FileSystem
### What changes were proposed in this pull request?

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

https://github.com/apache/spark/pull/31671 qualifies the warehouse at the beginning, which may fail Spark startup if something goes wrong, like the underlying FileSystem can't be initialized.

This PR falls back to the old behavior and leave the warehouse path unqualified if qualifying fails.

### Why are the changes needed?

Fix a regression. It's important to be always able to start Spark app (e.g. spark-shell), so that we can debug.

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

No

### How was this patch tested?

a new test case

Closes #32622 from cloud-fan/follow2.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-21 08:51:10 -07:00
itholic d2bdd6595e [SPARK-35025][SQL][PYTHON][DOCS] Move Parquet data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move Parquet data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for Parquet data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "Parquet Files" page
![Screen Shot 2021-05-21 at 1 35 08 PM](https://user-images.githubusercontent.com/44108233/119082866-e7375f00-ba39-11eb-9ade-a931a5957b34.png)

- Python
![Screen Shot 2021-05-21 at 1 38 27 PM](https://user-images.githubusercontent.com/44108233/119082879-eef70380-ba39-11eb-9e8e-ee50eed98dbe.png)

- Scala
![Screen Shot 2021-05-21 at 1 36 52 PM](https://user-images.githubusercontent.com/44108233/119082884-f1595d80-ba39-11eb-98d5-966657df65f7.png)

- Java
![Screen Shot 2021-05-21 at 1 37 19 PM](https://user-images.githubusercontent.com/44108233/119082888-f4544e00-ba39-11eb-8bf8-47ce78ec0b01.png)

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32161 from itholic/SPARK-34491.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:05:49 +09:00
itholic 419ddcb2a4 [SPARK-34494][SQL][DOCS] Move JSON data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move JSON data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for JSON data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "JSON Files" page
<img width="876" alt="Screen Shot 2021-05-20 at 8 48 27 PM" src="https://user-images.githubusercontent.com/44108233/118973662-ddb3e580-b9ac-11eb-987c-8139aa9c3fe2.png">

- Python
<img width="714" alt="Screen Shot 2021-04-16 at 5 04 11 PM" src="https://user-images.githubusercontent.com/44108233/114992491-ca0cef00-9ed5-11eb-9d0f-4de60d8b2516.png">

- Scala
<img width="726" alt="Screen Shot 2021-04-16 at 5 04 54 PM" src="https://user-images.githubusercontent.com/44108233/114992594-e315a000-9ed5-11eb-8bd3-af7e568fcfe1.png">

- Java
<img width="911" alt="Screen Shot 2021-04-16 at 5 06 11 PM" src="https://user-images.githubusercontent.com/44108233/114992751-10624e00-9ed6-11eb-888c-8668d3c74289.png">

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32204 from itholic/SPARK-35081.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:05:13 +09:00
itholic 0fe65b5365 [SPARK-35395][DOCS] Move ORC data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

This PR proposes move ORC data source options from Python, Scala and Java into a single page.

### Why are the changes needed?

So far, the documentation for ORC data source options is separated into different pages for each language API documents. However, this makes managing many options inconvenient, so it is efficient to manage all options in a single page and provide a link to that page in the API of each language.

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

Yes, the documents will be shown below after this change:

- "ORC Files" page
![Screen Shot 2021-05-21 at 2 07 14 PM](https://user-images.githubusercontent.com/44108233/119085078-f4564d00-ba3d-11eb-8990-3ba031d809da.png)

- Python
![Screen Shot 2021-05-21 at 2 06 46 PM](https://user-images.githubusercontent.com/44108233/119085097-00daa580-ba3e-11eb-8017-ac5a95a7c053.png)

- Scala
![Screen Shot 2021-05-21 at 2 06 09 PM](https://user-images.githubusercontent.com/44108233/119085135-164fcf80-ba3e-11eb-9cac-78dded523f38.png)

- Java
![Screen Shot 2021-05-21 at 2 06 30 PM](https://user-images.githubusercontent.com/44108233/119085125-118b1b80-ba3e-11eb-9434-f26612d7da13.png)

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32546 from itholic/SPARK-35395.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 18:03:57 +09:00
yi.wu 34284c0649 [SPARK-35454][SQL] One LogicalPlan can match multiple dataset ids
### What changes were proposed in this pull request?

Change the type of `DATASET_ID_TAG` from `Long` to `HashSet[Long]` to allow the logical plan to match multiple datasets.

### Why are the changes needed?

During the transformation from one Dataset to another Dataset, the DATASET_ID_TAG of logical plan won't change if the plan itself doesn't change:

b5241c97b1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala (L234-L237)

However, dataset id always changes even if the logical plan doesn't change:
b5241c97b1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala (L207-L208)

And this can lead to the mismatch between dataset's id and col's __dataset_id. E.g.,

```scala
  test("SPARK-28344: fail ambiguous self join - Dataset.colRegex as column ref") {
    // The test can fail if we change it to:
    // val df1 = spark.range(3).toDF()
    // val df2 = df1.filter($"id" > 0).toDF()
    val df1 = spark.range(3)
    val df2 = df1.filter($"id" > 0)

    withSQLConf(
      SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true",
      SQLConf.CROSS_JOINS_ENABLED.key -> "true") {
      assertAmbiguousSelfJoin(df1.join(df2, df1.colRegex("id") > df2.colRegex("id")))
    }
  }
```

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

No.

### How was this patch tested?

Added unit tests.

Closes #32616 from Ngone51/fix-ambiguous-join.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 16:15:17 +08:00
ulysses-you 83737852f0 [SPARK-35063][SQL][FOLLOWUP] Fix scala 2.13 error
### What changes were proposed in this pull request?

### Why are the changes needed?

Fix scala compile error.

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

No

### How was this patch tested?

Pass GA

Closes #32617 from ulysses-you/scala2-13.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 07:59:26 +00:00
Max Gekk 2b08070e79 [SPARK-35427][SQL][TESTS] Check the EXCEPTION rebase mode for Avro/Parquet
### What changes were proposed in this pull request?
Add tests to check the `EXCEPTION` rebase mode explicitly in the datasources:
- Parquet: `DATE` type and `TIMESTAMP`: `INT96`, `TIMESTAMP_MICROS`, `TIMESTAMP_MILLIS`
- Avro: `DATE` type and `TIMESTAMP`: `timestamp-millis` and `timestamp-micros`.

### Why are the changes needed?
1. To improve test coverage
2. The `EXCEPTION` rebase mode should be checked independently from the default settings.

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

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly *AvroV2Suite"
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite"
```

Closes #32574 from MaxGekk/test-rebase-exception.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 06:18:06 +00:00
gengjiaan c740c097e0 [SPARK-35063][SQL] Group exception messages in sql/catalyst
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32478 from beliefer/SPARK-35063.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 06:15:26 +00:00
Takeshi Yamamuro 1a923f5319 [SPARK-35479][SQL] Format PartitionFilters IN strings in scan nodes
### What changes were proposed in this pull request?

This PR proposes to format strings correctly for `PushedFilters`. For example, `explain()` for a query below prints `v in (array('a'))` as `PushedFilters: [In(v, [WrappedArray(a)])]`;

```
scala> sql("create table t (v array<string>) using parquet")
scala> sql("select * from t where v in (array('a'), null)").explain()
== Physical Plan ==
*(1) Filter v#4 IN ([a],null)
+- FileScan parquet default.t[v#4] Batched: false, DataFilters: [v#4 IN ([a],null)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-3.1.1-bin-hadoop2.7/spark-warehouse/t], PartitionFilters: [], PushedFilters: [In(v, [WrappedArray(a),null])], ReadSchema: struct<v:array<string>>
```
This PR makes `explain()` print it as `PushedFilters: [In(v, [[a]])]`;
```
scala> sql("select * from t where v in (array('a'), null)").explain()
== Physical Plan ==
*(1) Filter v#4 IN ([a],null)
+- FileScan parquet default.t[v#4] Batched: false, DataFilters: [v#4 IN ([a],null)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-3.1.1-bin-hadoop2.7/spark-warehouse/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
```
NOTE: This PR includes a bugfix caused by #32577 (See the cloud-fan comment: https://github.com/apache/spark/pull/32577/files#r636108150).

### Why are the changes needed?

To improve explain strings.

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

Yes, this PR improves the explain strings for pushed-down filters.

### How was this patch tested?

Added tests in `SQLQueryTestSuite`.

Closes #32615 from maropu/ExplainPartitionFilters.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 05:45:45 +00:00
yi.wu e1296eab5f [SPARK-35445][SQL] Reduce the execution time of DeduplicateRelations
### What changes were proposed in this pull request?

This PR reduces the execution time of `DeduplicateRelations` by:

1) use `Set` instead `Seq` to check duplicate relations

2) avoid plan output traverse and attribute rewrites when there are no changes in the children plan

### Why are the changes needed?

Rule `DeduplicateRelations` is slow.

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

No.

### How was this patch tested?

Run `TPCDSQuerySuite` and checked the run time of `DeduplicateRelations`. The time has been reduced by 77.9% after this PR.

Closes #32590 from Ngone51/improve-dedup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-21 13:25:37 +08:00
Kent Yao 2e9936db93 [SPARK-35456][CORE] Print the invalid value in config validation error message
### What changes were proposed in this pull request?

Print the invalid value in config validation error message for `checkValue` just like `checkValues`

### Why are the changes needed?

Invalid configuration values may come in many ways, this PR can help different kinds of users or developers to identify what the config the error is related to

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

yes, but only error msg
### How was this patch tested?

yes, modified tests

Closes #32600 from yaooqinn/SPARK-35456.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 14:22:29 +09:00
Vinod KC bdd8e1dbb1 [SPARK-28551][SQL] CTAS with LOCATION should not allow to a non-empty directory
### What changes were proposed in this pull request?

CTAS with location clause acts as an insert overwrite. This can cause problems when there are subdirectories within a location directory.
This causes some users to accidentally wipe out directories with very important data. We should not allow CTAS with location to a non-empty directory.

### Why are the changes needed?

Hive already handled this scenario: HIVE-11319

Steps to reproduce:

```scala
sql("""create external table  `demo_CTAS`( `comment` string) PARTITIONED BY (`col1` string, `col2` string) STORED AS parquet location '/tmp/u1/demo_CTAS'""")
sql("""INSERT OVERWRITE TABLE demo_CTAS partition (col1='1',col2='1') VALUES ('abc')""")
sql("select* from demo_CTAS").show
sql("""create table ctas1 location '/tmp/u2/ctas1' as select * from demo_CTAS""")
sql("select* from ctas1").show
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```

Before the fix: Both create table operations will succeed. But values in table ctas1 will be replaced by ctas2 accidentally.

After the fix: `create table ctas2...` will throw `AnalysisException`:

```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

### Does this PR introduce _any_ user-facing change?
Yes, if the location directory is not empty, CTAS with location will throw AnalysisException

```
sql("""create table ctas2 location '/tmp/u2' as select * from demo_CTAS""")
```
```
org.apache.spark.sql.AnalysisException: CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory /tmp/u2 . To allow overwriting the existing non-empty directory, set 'spark.sql.legacy.allowNonEmptyLocationInCTAS' to true.
```

`CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, need to  set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`. , default value is `false`.
Updated SQL migration guide.

### How was this patch tested?
Test case added in SQLQuerySuite.scala

Closes #32411 from vinodkc/br_fixCTAS_nonempty_dir.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 06:13:18 +00:00
Cheng Su 586caae3cc [SPARK-35438][SQL][DOCS] Minor documentation fix for window physical operator
### What changes were proposed in this pull request?

As title. Fixed two places where the documentation for window operator has some error.

### Why are the changes needed?

Help people read code for window operator more easily in the future.

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

No.

### How was this patch tested?

Existing tests.

Closes #32585 from c21/minor-doc.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-20 08:47:19 +09:00
Andy Grove 52e3cf9ff5 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use
### What changes were proposed in this pull request?
AQE has an optimization where it attempts to reuse compatible exchanges but it does not take into account whether the exchanges are columnar or not, resulting in incorrect reuse under some circumstances.

This PR simply changes the key used to lookup cached stages. It now uses the canonicalized form of the new query stage (potentially created by a plugin) rather than using the canonicalized form of the original exchange.

### Why are the changes needed?
When using the [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) we sometimes see a new query stage correctly create a row-based exchange and then Spark replaces it with a cached columnar exchange, which is not compatible, and this causes queries to fail.

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

### How was this patch tested?
The patch has been tested with the query that highlighted this issue. I looked at writing unit tests for this but it would involve implementing a mock columnar exchange in the tests so would be quite a bit of work. If anyone has ideas on other ways to test this I am happy to hear them.

Closes #32195 from andygrove/SPARK-35093.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-05-19 07:45:26 -05:00
shahid 12142130cd [SPARK-35362][SQL] Update null count in the column stats for UNION operator stats estimation
### What changes were proposed in this pull request?
Updating column stats for Union operator stats estimation
### Why are the changes needed?
This is a followup PR to update the null count also in the Union stats operator estimation. https://github.com/apache/spark/pull/30334

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

### How was this patch tested?
Updated UTs, manual testing

Closes #32494 from shahidki31/shahid/updateNullCountForUnion.

Lead-authored-by: shahid <shahidki31@gmail.com>
Co-authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-19 21:23:19 +09:00
Kousuke Saruta 9283bebbbd [SPARK-35418][SQL] Add sentences function to functions.{scala,py}
### What changes were proposed in this pull request?

This PR adds `sentences`, a string function, which is present as of `2.0.0` but missing in `functions.{scala,py}`.

### Why are the changes needed?

This function can be only used from SQL for now.
It's good if we can use this function from Scala/Python code as well as SQL.

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

Yes. Users can use this function from Scala and Python.

### How was this patch tested?

New test.

Closes #32566 from sarutak/sentences-function.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-19 20:07:28 +09:00
shahid 46f7d780d3 [SPARK-35368][SQL] Update histogram statistics for RANGE operator for stats estimation
### What changes were proposed in this pull request?
Update histogram statistics for RANGE operator stats estimation
### Why are the changes needed?
If histogram optimization is enabled, this statistics can be used in various cost based optimizations.

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

### How was this patch tested?
Added UTs. Manual test.

Closes #32498 from shahidki31/shahid/histogram.

Lead-authored-by: shahid <shahidki31@gmail.com>
Co-authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-19 16:49:32 +09:00
Yuzhou Sun a72d05c7e6 [SPARK-35106][CORE][SQL] Avoid failing rename caused by destination directory not exist
### What changes were proposed in this pull request?

1. In HadoopMapReduceCommitProtocol, create parent directory before renaming custom partition path staging files
2. In InMemoryCatalog and HiveExternalCatalog, create new partition directory before renaming old partition path
3. Check return value of FileSystem#rename, if false, throw exception to avoid silent data loss cause by rename failure
4. Change DebugFilesystem#rename behavior to make it match HDFS's behavior (return false without rename when dst parent directory not exist)

### Why are the changes needed?

Depends on FileSystem#rename implementation, when destination directory does not exist, file system may
1. return false without renaming file nor throwing exception (e.g. HDFS), or
2. create destination directory, rename files, and return true (e.g. LocalFileSystem)

In the first case above, renames in HadoopMapReduceCommitProtocol for custom partition path will fail silently if the destination partition path does not exist. Failed renames can happen when
1. dynamicPartitionOverwrite == true, the custom partition path directories are deleted by the job before the rename; or
2. the custom partition path directories do not exist before the job; or
3. something else is wrong when file system handle `rename`

The renames in MemoryCatalog and HiveExternalCatalog for partition renaming also have similar issue.

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

No

### How was this patch tested?

Modified DebugFilesystem#rename, and added new unit tests.

Without the fix in src code, five InsertSuite tests and one AlterTableRenamePartitionSuite test failed:
InsertSuite.SPARK-20236: dynamic partition overwrite with custom partition path (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: insert overwrite with custom partition path
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: dynamic partition overwrite with custom partition path
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 1 ==
!struct<>                   struct<i:int,part1:int,part2:int>
 [1,1,1]                    [1,1,1]
![1,1,2]
```

InsertSuite.SPARK-35106: Throw exception when rename custom partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

InsertSuite.SPARK-35106: Throw exception when rename dynamic partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

AlterTableRenamePartitionSuite.ALTER TABLE .. RENAME PARTITION V1: multi part partition (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
 struct<>                   struct<>
![3,123,3]
```

Closes #32530 from YuzhouSun/SPARK-35106.

Authored-by: Yuzhou Sun <yuzhosun@amazon.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 15:46:27 +08:00
Kousuke Saruta 0b3758e8cd [SPARK-35421][SS] Remove redundant ProjectExec from streaming queries with V2Relation
### What changes were proposed in this pull request?

This PR fixes an issue that streaming queries with V2Relation can have redundant `ProjectExec` in its physical plan.
You can easily reproduce this issue with the following code.
```
import org.apache.spark.sql.streaming.Trigger

val query = spark.
  readStream.
  format("rate").
  option("rowsPerSecond", 1000).
  option("rampUpTime", "10s").
  load().
  selectExpr("timestamp", "100",  "value").
  writeStream.
  format("console").
  trigger(Trigger.ProcessingTime("5 seconds")).
  // trigger(Trigger.Continuous("5 seconds")). // You can reproduce with continuous processing too.
  outputMode("append").
  start()
```
The plan tree is here.
![ss-before](https://user-images.githubusercontent.com/4736016/118454996-ec439800-b733-11eb-8cd8-ed8af73a91b8.png)

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

I run the same code above and get the following plan tree.
![ss-after](https://user-images.githubusercontent.com/4736016/118455755-1bf2a000-b734-11eb-999e-4b8c19ad34d7.png)

Closes #32570 from sarutak/fix-redundant-projectexec.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 06:19:01 +00:00
yangjie01 b1493d82dd [SPARK-35398][SQL] Simplify the way to get classes from ClassBodyEvaluator in CodeGenerator.updateAndGetCompilationStats method
### What changes were proposed in this pull request?
SPARK-35253 upgraded janino from 3.0.16 to 3.1.4, `ClassBodyEvaluator` provides the `getBytecodes` method to get
the mapping from `ClassFile#getThisClassName` to `ClassFile#toByteArray` directly in this version and we don't need to get this variable by reflection api anymore.

So the main purpose of this pr is simplify the way to get `bytecodes` from `ClassBodyEvaluator` in `CodeGenerator#updateAndGetCompilationStats` method.

### Why are the changes needed?
Code simplification.

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action

- Manual test:

1. Define a code fragment to be tested, for example:
```
    val codeBody = s"""
        public java.lang.Object generate(Object[] references) {
          return new TestMetricCode(references);
        }

        class TestMetricCode {

          public TestMetricCode(Object[] references) {
          }

          public long sumOfSquares(long left, long right) {
            return left * left + right * right;
          }
        }
      """
```
2. Create a `ClassBodyEvaluator` and `cook` the `codeBody` as above, the process of creating `ClassBodyEvaluator` can extract from `CodeGenerator#doCompile` method.

3. Get `bytecodes` using `ClassBodyEvaluator#getBytecodes` api(after this pr) and reflection api(before this pr) respectively, then assert that they are the same. If the `bytecodes` not changed, we can be sure that metrics state will not change. The test code example as follows:
```
    import scala.collection.JavaConverters._
    val bytecodesFromApi = evaluator.getBytecodes.asScala
    val bytecodesFromReflectionApi = {
      val scField = classOf[ClassBodyEvaluator].getDeclaredField("sc")
      scField.setAccessible(true)
      val compiler = scField.get(evaluator).asInstanceOf[SimpleCompiler]
      val loader = compiler.getClassLoader.asInstanceOf[ByteArrayClassLoader]
      val classesField = loader.getClass.getDeclaredField("classes")
      classesField.setAccessible(true)
      classesField.get(loader).asInstanceOf[java.util.Map[String, Array[Byte]]].asScala
    }

    assert(bytecodesFromApi == bytecodesFromReflectionApi)
```

Closes #32536 from LuciferYang/SPARK-35253-FOLLOWUP.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-19 13:03:35 +09:00
Tengfei Huang 9804f07c17 [SPARK-35411][SQL] Add essential information while serializing TreeNode to json
### What changes were proposed in this pull request?
Write out Seq of product objects which contain TreeNode, to avoid the cases as described in https://issues.apache.org/jira/browse/SPARK-35411 that essential information will be ignored and just written out as null values. These information are necessary to understand the query plans.

### Why are the changes needed?
Information like cteRelations in With node, and branches in CaseWhen expression are necessary to understand the query plans, they should be written out to the result json string.

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

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

Closes #32557 from ivoson/plan-json-fix.

Authored-by: Tengfei Huang <tengfei.h@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 23:20:12 +08:00
Takeshi Yamamuro 746d80d87a [SPARK-35422][SQL] Fix plan-printing issues to pass the TPCDS plan stability tests in Scala v2.13
### What changes were proposed in this pull request?

To pass the TPCDS-related plan stability tests in scala-2.13, this PR proposes to fix two things below;
 - (1) Sorts elements in the predicate `InSet` and the source filter `In` for printing their nodes.
 - (2) Formats nested collection elements (`Seq`, `Array`, and `Set`) recursively in `TreeNode.argString`.

As for (1), it seems v2.12/v2.13 prints `Set` elements with a different order, so we need to sort them explicitly. As for (2), the `Seq` implementation is different between v2.12/v2.13, so we need to format nested  `Seq` elements correctly to hide the name of its implementation (See an example below);
```
 (74) Expand [codegen id : 20]
 Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45]
-Arguments: [ArrayBuffer(sales#41, returns#42, ...    <-- scala-2.12
+Arguments: [Vector(sales#41, returns#42, ...         <-- scala-2.13

+Arguments: [[(sales#41, returns#42, ...              <--  the proposed fix to hide the name of its implementation
```

### Why are the changes needed?

To pass the tests in Scala v2.13.

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

Yes, this fix changes query explain strings.

### How was this patch tested?

Manually checked.

Closes #32577 from maropu/FixTPCDSTestIssueInScala213.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 08:50:00 +00:00
Chao Sun 44d762abc6 [SPARK-35389][SQL] V2 ScalarFunction should support magic method with null arguments
### What changes were proposed in this pull request?

When creating `Invoke` and `StaticInvoke` for `ScalarFunction`'s magic method, set `propagateNull` to false.

### Why are the changes needed?

When `propgagateNull` is true (which is the default value), `Invoke` and `StaticInvoke` will return null if any of the argument is null. For scalar function this is incorrect, as we should leave the logic to function implementation instead.

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

Yes. Now null arguments shall be properly handled with magic method.

### How was this patch tested?

Added new tests.

Closes #32553 from sunchao/SPARK-35389.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 08:45:55 +00:00
Cheng Su cce0048c78 [SPARK-35351][SQL] Add code-gen for left anti sort merge join
### What changes were proposed in this pull request?

As title. This PR is to add code-gen support for LEFT ANTI sort merge join. The main change is to extract `loadStreamed` in `SortMergeJoinExec.doProduce()`. That is to set all columns values for streamed row, when the streamed row has no output row.

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"))
val df2 = spark.range(4).select($"id".as("k2"))
df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti")
```

Example generated code:

```
== Subtree 5 / 5 (maxMethodCodeSize:296; maxConstantPoolSize:156(0.24% used); numInnerClasses:0) ==
*(5) Project [id#0L AS k1#2L]
+- *(5) SortMergeJoin [id#0L], [k2#6L], LeftAnti
   :- *(2) Sort [id#0L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#0L, 5), ENSURE_REQUIREMENTS, [id=#27]
   :     +- *(1) Range (0, 10, step=1, splits=2)
   +- *(4) Sort [k2#6L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(k2#6L, 5), ENSURE_REQUIREMENTS, [id=#33]
         +- *(3) Project [id#4L AS k2#6L]
            +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(1, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     smj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */
/* 032 */   }
/* 033 */
/* 034 */   private boolean findNextJoinRows(
/* 035 */     scala.collection.Iterator streamedIter,
/* 036 */     scala.collection.Iterator bufferedIter) {
/* 037 */     smj_streamedRow_0 = null;
/* 038 */     int comp = 0;
/* 039 */     while (smj_streamedRow_0 == null) {
/* 040 */       if (!streamedIter.hasNext()) return false;
/* 041 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 042 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 043 */       if (false) {
/* 044 */         if (!smj_matches_0.isEmpty()) {
/* 045 */           smj_matches_0.clear();
/* 046 */         }
/* 047 */         return false;
/* 048 */
/* 049 */       }
/* 050 */       if (!smj_matches_0.isEmpty()) {
/* 051 */         comp = 0;
/* 052 */         if (comp == 0) {
/* 053 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 054 */         }
/* 055 */
/* 056 */         if (comp == 0) {
/* 057 */           return true;
/* 058 */         }
/* 059 */         smj_matches_0.clear();
/* 060 */       }
/* 061 */
/* 062 */       do {
/* 063 */         if (smj_bufferedRow_0 == null) {
/* 064 */           if (!bufferedIter.hasNext()) {
/* 065 */             smj_value_3 = smj_value_0;
/* 066 */             return !smj_matches_0.isEmpty();
/* 067 */           }
/* 068 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 069 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 070 */           if (false) {
/* 071 */             smj_bufferedRow_0 = null;
/* 072 */             continue;
/* 073 */           }
/* 074 */           smj_value_2 = smj_value_1;
/* 075 */         }
/* 076 */
/* 077 */         comp = 0;
/* 078 */         if (comp == 0) {
/* 079 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 080 */         }
/* 081 */
/* 082 */         if (comp > 0) {
/* 083 */           smj_bufferedRow_0 = null;
/* 084 */         } else if (comp < 0) {
/* 085 */           if (!smj_matches_0.isEmpty()) {
/* 086 */             smj_value_3 = smj_value_0;
/* 087 */             return true;
/* 088 */           } else {
/* 089 */             return false;
/* 090 */           }
/* 091 */         } else {
/* 092 */           if (smj_matches_0.isEmpty()) {
/* 093 */             smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 094 */           }
/* 095 */
/* 096 */           smj_bufferedRow_0 = null;
/* 097 */         }
/* 098 */       } while (smj_streamedRow_0 != null);
/* 099 */     }
/* 100 */     return false; // unreachable
/* 101 */   }
/* 102 */
/* 103 */   protected void processNext() throws java.io.IOException {
/* 104 */     while (smj_streamedInput_0.hasNext()) {
/* 105 */       findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0);
/* 106 */
/* 107 */       long smj_value_4 = -1L;
/* 108 */       smj_value_4 = smj_streamedRow_0.getLong(0);
/* 109 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 110 */
/* 111 */       boolean wholestagecodegen_hasOutputRow_0 = false;
/* 112 */
/* 113 */       while (!wholestagecodegen_hasOutputRow_0 && smj_iterator_0.hasNext()) {
/* 114 */         InternalRow smj_bufferedRow_1 = (InternalRow) smj_iterator_0.next();
/* 115 */
/* 116 */         wholestagecodegen_hasOutputRow_0 = true;
/* 117 */       }
/* 118 */
/* 119 */       if (!wholestagecodegen_hasOutputRow_0) {
/* 120 */         // load all values of streamed row, because the values not in join condition are not
/* 121 */         // loaded yet.
/* 122 */
/* 123 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 124 */
/* 125 */         // common sub-expressions
/* 126 */
/* 127 */         smj_mutableStateArray_0[1].reset();
/* 128 */
/* 129 */         smj_mutableStateArray_0[1].write(0, smj_value_4);
/* 130 */         append((smj_mutableStateArray_0[1].getRow()).copy());
/* 131 */
/* 132 */       }
/* 133 */       if (shouldStop()) return;
/* 134 */     }
/* 135 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 136 */   }
/* 137 */
/* 138 */ }
```

### Why are the changes needed?

Improve the query CPU performance.

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala`, and existed unit test in `ExistenceJoinSuite.scala`.

Closes #32547 from c21/smj-left-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-18 16:56:45 +09:00
Takeshi Yamamuro 3b859a16c0 [SPARK-35431][SQL][TESTS] Sort elements generated by collect_set in SQLQueryTestSuite
### What changes were proposed in this pull request?

To pass `subquery/scalar-subquery/scalar-subquery-select.sql` (`SQLQueryTestSuite`) in Scala v2.13,  this PR proposes to change the aggregate expr of a test query in the file from `collect_set(...)` to `sort_array(collect_set(...))` because `collect_set` depends on the `mutable.HashSet` implementation and elements in the set are printed in a different order in Scala v2.12/v2.13.

### Why are the changes needed?

To pass the test in Scala v2.13.

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

No, dev-only.

### How was this patch tested?

Manually checked.

Closes #32578 from maropu/FixSQLTestIssueInScala213.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-17 22:51:32 -07:00
Hyukjin Kwon 747fe7282c [SPARK-35419][PYTHON] Enable spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled by default
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/30309 added a configuration (disabled by default) that simplifies the error messages from Python UDFS, which removed internal stacktrace from Python workers:

```python
from pyspark.sql.functions import udf; spark.range(10).select(udf(lambda x: x/0)("id")).collect()
```

**Before**

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../python/pyspark/sql/utils.py", line 127, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 605, in main
    process()
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 597, in process
    serializer.dump_stream(out_iter, outfile)
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 223, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 141, in dump_stream
    for obj in iterator:
  File "/.../python/lib/pyspark.zip/pyspark/serializers.py", line 212, in _batched
    for item in iterator:
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in mapper
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 450, in <genexpr>
    result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in udfs)
  File "/.../python/lib/pyspark.zip/pyspark/worker.py", line 90, in <lambda>
    return lambda *a: f(*a)
  File "/.../python/lib/pyspark.zip/pyspark/util.py", line 107, in wrapper
    return f(*args, **kwargs)
  File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```

**After**

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../python/pyspark/sql/dataframe.py", line 427, in show
    print(self._jdf.showString(n, 20, vertical))
  File "/.../python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in __call__
  File "/.../python/pyspark/sql/utils.py", line 127, in deco
    raise_from(converted)
  File "<string>", line 3, in raise_from
pyspark.sql.utils.PythonException:
  An exception was thrown from Python worker in the executor:
Traceback (most recent call last):
  File "<stdin>", line 1, in <lambda>
ZeroDivisionError: division by zero
```

Note that the traceback (`return f(*args, **kwargs)`) is almost always same - I would say more than 99%. For 1% case, we can guide developers to enable this configuration for further debugging.

In Databricks, it has been enabled for around 6 months, and I have had zero negative feedback on it.

### Why are the changes needed?

To show simplified exception messages to end users.

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

Yes, it will hide the internal Python worker traceback.

### How was this patch tested?

Existing test cases should cover.

Closes #32569 from HyukjinKwon/SPARK-35419.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-18 12:27:09 +09:00
Chao Sun a60c36458d [SPARK-34981][SQL][TESTS][FOLLOWUP] Fix test failure under Scala 2.13
### What changes were proposed in this pull request?

Fix test failure under Scala 2.13 by making test `ScalaFunction` `StrLenMagic` public.

### Why are the changes needed?

A few tests are failing when using Scala 2.13 with error message like the following:
```
[info]   Cause: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 35, Column 121: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 35, Column 121: No a
pplicable constructor/method found for actual parameters "org.apache.spark.unsafe.types.UTF8String"; candidates are: "public int org.apache.spark.sql.connector.DataSourceV2FunctionSuite$StrLenMagic$.invoke(org.apache.spark.
unsafe.types.UTF8String)"
[info]   at org.apache.spark.sql.errors.QueryExecutionErrors$.compilerError(QueryExecutionErrors.scala:387)
[info]   at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1415)
[info]   at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1501)
```

This seems to be caused by the fact that the `StrLenMagic` is using `private` scope. After removing the `private` keyword the tests are now passing.

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

No

### How was this patch tested?

```
$ dev/change-scala-version.sh 2.13
$ build/sbt "sql/testOnly *.DataSourceV2FunctionSuite" -Pscala-2.13
```

Closes #32575 from sunchao/SPARK-34981-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-17 16:17:01 -07:00
fhygh 3a3f8ca6f4 [SPARK-35359][SQL] Insert data with char/varchar datatype will fail when data length exceed length limitation
### What changes were proposed in this pull request?
This PR is used to fix this bug:

```
set spark.sql.legacy.charVarcharAsString=true;
create table chartb01(a char(3));
insert into chartb01 select 'aaaaa';
```

here we expect the data of table chartb01 is 'aaa', but it runs failed.

### Why are the changes needed?
Improve backward compatibility

```
spark-sql>
         > create table tchar01(col char(2)) using parquet;
Time taken: 0.767 seconds
spark-sql>
         > insert into tchar01 select 'aaa';
ERROR | Executor task launch worker for task 0.0 in stage 0.0 (TID 0) | Aborting task | org.apache.spark.util.Utils.logError(Logging.scala:94)
java.lang.RuntimeException: Exceeds char/varchar type length limitation: 2
        at org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils.trimTrailingSpaces(CharVarcharCodegenUtils.java:31)
        at org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils.charTypeWriteSideCheck(CharVarcharCodegenUtils.java:44)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.project_doConsume_0$(Unknown Source)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
        at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:279)
        at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1500)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:288)
        at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:212)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1466)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
        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)
```

### Does this PR introduce _any_ user-facing change?
No (the legacy config is false by default).

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

Closes #32501 from fhygh/master.

Authored-by: fhygh <283452027@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-18 00:13:40 +08:00
Wenchen Fan 3b63f32601 [SPARK-35400][SQL] Simplify getOuterReferences and improve error message for correlated subquery
### What changes were proposed in this pull request?

Spark doesn't support aggregate functions with mixed outer and local references. This PR applies this check earlier to fail with a clear error message instead of some weird ones, and simplifies the related code in `SubExprUtils.getOuterReferences`. This PR also refines the error message a bit.

### Why are the changes needed?

better error message

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

no

### How was this patch tested?

updated tests

Closes #32503 from cloud-fan/try.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-17 14:13:44 +00:00
Jungtaek Lim 7c13636be3 [SPARK-34888][SS] Introduce UpdatingSessionIterator adjusting session window on elements
Introduction: this PR is a part of SPARK-10816 (`EventTime based sessionization (session window)`). Please refer #31937 to see the overall view of the code change. (Note that code diff could be diverged a bit.)

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

This PR introduces UpdatingSessionsIterator, which analyzes neighbor elements and adjust session information on elements.

UpdatingSessionsIterator calculates and updates the session window for each element in the given iterator, which makes elements in the same session window having same session spec. Downstream can apply aggregation to finally merge these elements bound to the same session window.

UpdatingSessionsIterator works on the precondition that given iterator is sorted by "group keys + start time of session window", and the iterator still retains the characteristic of the sort.

UpdatingSessionsIterator copies the elements to safely update on each element, as well as buffers elements which are bound to the same session window. Due to such overheads, MergingSessionsIterator which will be introduced via SPARK-34889 should be used whenever possible.

This PR also introduces UpdatingSessionsExec which is the physical node on leveraging UpdatingSessionsIterator to sort the input rows and updates session information on input rows.

### Why are the changes needed?

This part is a one of required on implementing SPARK-10816.

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

No.

### How was this patch tested?

New test suite added.

Closes #31986 from HeartSaVioR/SPARK-34888-SPARK-10816-PR-31570-part-1.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-17 21:05:49 +09:00
Yuming Wang fb9316388a [SPARK-32792][SQL][FOLLOWUP] Fix conflict with SPARK-34661
### What changes were proposed in this pull request?

This fixes the compilation error due to the logical conflicts between https://github.com/apache/spark/pull/31776 and https://github.com/apache/spark/pull/29642 .

### Why are the changes needed?

To recover compilation.

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

No.

### How was this patch tested?

Closes #32568 from wangyum/HOT-FIX.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 22:12:52 -07:00
Yuming Wang d2d1f0b580 [SPARK-32792][SQL] Improve Parquet In filter pushdown
### What changes were proposed in this pull request?

Support push down `GreaterThanOrEqual` minimum value and `LessThanOrEqual` maximum value for Parquet  when [sources.In](a744fea3be/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala (L162-L181))'s values exceeds `spark.sql.optimizer.inSetRewriteMinMaxThreshold`. For example:

```sql
SELECT * FROM t WHERE id IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 15)
```

We will push down `id >= 1 and id <= 15`.

Impala also has this improvement: https://issues.apache.org/jira/browse/IMPALA-3654

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test, [manual test](https://github.com/apache/spark/pull/29642#issuecomment-743109098) and benchmark test.

Before this PR:
```
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5995           6026          53          2.6         381.2       1.0X
Parquet Vectorized (Pushdown)                                      423            440          11         37.2          26.9      14.2X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5767           5887         154          2.7         366.7       1.0X
Parquet Vectorized (Pushdown)                                      419            428           6         37.6          26.6      13.8X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5764           5857          96          2.7         366.4       1.0X
Parquet Vectorized (Pushdown)                                      408            419           9         38.6          25.9      14.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5895           5949          41          2.7         374.8       1.0X
Parquet Vectorized (Pushdown)                                      5908           5986         114          2.7         375.6       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5893           5988         106          2.7         374.7       1.0X
Parquet Vectorized (Pushdown)                                      5875           5939          57          2.7         373.5       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5891           5954          42          2.7         374.5       1.0X
Parquet Vectorized (Pushdown)                                      5901           5976          99          2.7         375.2       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6128           6158          40          2.6         389.6       1.0X
Parquet Vectorized (Pushdown)                                       6145           6190          37          2.6         390.7       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6142           6217          64          2.6         390.5       1.0X
Parquet Vectorized (Pushdown)                                       6149           6235          90          2.6         391.0       1.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6148           6218          64          2.6         390.9       1.0X
Parquet Vectorized (Pushdown)                                       6145           6177          30          2.6         390.7       1.0X
```

After this PR:
```
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5745           5768          28          2.7         365.2       1.0X
Parquet Vectorized (Pushdown)                                      401            412          12         39.2          25.5      14.3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5796           5861          61          2.7         368.5       1.0X
Parquet Vectorized (Pushdown)                                      417            482          37         37.7          26.5      13.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 10, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
---------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                5754           5777          20          2.7         365.8       1.0X
Parquet Vectorized (Pushdown)                                      408            418           9         38.6          25.9      14.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5878           5915          40          2.7         373.7       1.0X
Parquet Vectorized (Pushdown)                                       929            940          10         16.9          59.1       6.3X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5886           5917          29          2.7         374.2       1.0X
Parquet Vectorized (Pushdown)                                      3091           3114          20          5.1         196.5       1.9X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 100, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                 5913           5948          48          2.7         375.9       1.0X
Parquet Vectorized (Pushdown)                                      5330           5427          98          3.0         338.9       1.1X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 10):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6147           6228          72          2.6         390.8       1.0X
Parquet Vectorized (Pushdown)                                       1023           1029           4         15.4          65.1       6.0X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 50):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6164           6224          47          2.6         391.9       1.0X
Parquet Vectorized (Pushdown)                                       3332           3360          45          4.7         211.9       1.8X

Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
InSet -> InFilters (values count: 2000, distribution: 90):  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-----------------------------------------------------------------------------------------------------------------------------------------
Parquet Vectorized                                                  6154           6192          38          2.6         391.3       1.0X
Parquet Vectorized (Pushdown)                                       5588           5679          92          2.8         355.3       1.1X
```

Closes #29642 from wangyum/SPARK-32792.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <yumwang@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 21:20:52 -07:00
Takeshi Yamamuro 2eef2f9035 [SPARK-35412][SQL] Fix a bug in groupBy of year-month/day-time intervals
### What changes were proposed in this pull request?

To fix a bug below in groupBy of year-month/day-time intervals, this PR proposes to make `HashMapGenerator` handle the two types for hash-aggregates;
```
scala> Seq(java.time.Duration.ofDays(1)).toDF("a").groupBy("a").count().show()
scala.MatchError: DayTimeIntervalType (of class org.apache.spark.sql.types.DayTimeIntervalType$)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.genComputeHash(HashMapGenerator.scala:159)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.$anonfun$generateHashFunction$1(HashMapGenerator.scala:102)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.immutable.List.map(List.scala:298)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.genHashForKeys$1(HashMapGenerator.scala:99)
  at org.apache.spark.sql.execution.aggregate.HashMapGenerator.generateHashFunction(HashMapGenerator.scala:111)
```

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added a unit test.

Closes #32560 from maropu/FixIntervalIssue.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 10:51:32 -07:00
Cheng Su 5c1567ba97 [SPARK-35363][SQL][FOLLOWUP] Use fresh name for findNextJoinRows instead of hardcoding it
### What changes were proposed in this pull request?

This is a followup from discussion in https://github.com/apache/spark/pull/32495#discussion_r632283178 . The hardcoded function name `findNextJoinRows` is not a real problem now as we always do code generation for SMJ's children separately. But this change is to make it future proof in case this assumption changed in the future.

### Why are the changes needed?

Fix the potential reliability issue.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32548 from c21/smj-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-16 10:49:31 -07:00
yangjie01 7ca0a0910f [SPARK-34661][SQL] Clean up OriginalType and DecimalMetadata usage in Parquet related code
### What changes were proposed in this pull request?
`OriginalType` and `DecimalMetadata` has been marked as `Deprecated` in new Parquet code.

`Apache Parquet` suggest us replace `OriginalType` with `LogicalTypeAnnotation` and replace `DecimalMetadata` with `DecimalLogicalTypeAnnotation`,  so the main change of this pr is clean up these deprecated usages in Parquet related code.

### Why are the changes needed?
Cleanup deprecated api usage.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31776 from LuciferYang/cleanup-parquet-dep-api.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-16 09:03:26 -05:00
Yuming Wang 520a355516 [SPARK-35286][SQL] Replace SessionState.start with SessionState.setCurrentSessionState
### What changes were proposed in this pull request?

This PR replaces `SessionState.start` with `shim.setCurrentSessionState/SessionState.setCurrentSessionState`.

### Why are the changes needed?

To avoid [SessionState.createSessionDirs](https://github.com/apache/hive/blob/rel/release-2.3.8/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L652-L696) creating too many directories and Spark SQL do not need it:
![image](https://user-images.githubusercontent.com/5399861/116766834-28ea7080-aa5f-11eb-85ff-07bcaee444e5.png)

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

No.

### How was this patch tested?

Existing test.

Closes #32410 from wangyum/setCurrentSessionState.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-05-16 18:39:15 +08:00
QuangHuyViettel 9789ee84e4 [SPARK-32484][SQL] Fix log info BroadcastExchangeExec.scala
### What changes were proposed in this pull request?
Fix log info in BroadcastExchangeExec.scala

### Why are the changes needed?
Log info s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB")  is not accurate info , because  8GB  is not accurate.
### Does this PR introduce _any_ user-facing change?
yes

### How was this patch tested?
no

Closes #32544 from LittleCuteBug/SPARK-32484.

Authored-by: QuangHuyViettel <quanghuynguyen236@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-15 13:08:42 -05:00
Chao Sun a8032e7efa [SPARK-35384][SQL][FOLLOWUP] Move HashMap.get out of InvokeLike.invoke
### What changes were proposed in this pull request?

Move hash map lookup operation out of `InvokeLike.invoke` since it doesn't depend on the input.

### Why are the changes needed?

We shouldn't need to look up the hash map for every input row evaluated by `InvokeLike.invoke` since it doesn't depend on input. This could speed up the performance a bit.

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

No

### How was this patch tested?

Existing tests.

Closes #32532 from sunchao/SPARK-35384-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-14 14:00:39 -07:00
yi.wu 94bd480761 [SPARK-35206][TESTS][SQL] Extract common used get project path into a function in SparkFunctionSuite
### What changes were proposed in this pull request?

Add a common functions `getWorkspaceFilePath` (which prefixed with spark home) to `SparkFunctionSuite`, and applies these the function to where they're extracted from.

### Why are the changes needed?

Spark sql has test suites to read resources when running tests. The way of getting the path of resources is commonly used in different suites. We can extract them into a function to ease the code maintenance.

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

No.

### How was this patch tested?

Pass existing tests.

Closes #32315 from Ngone51/extract-common-file-path.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-14 22:17:50 +08:00
ulysses-you 6218bc5036 [SPARK-35332][SQL][FOLLOWUP] Refine wrong comment
### What changes were proposed in this pull request?

Refine comment in `CacheManager`.

### Why are the changes needed?

Avoid misleading developer.

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

No.

### How was this patch tested?

Not needed.

Closes #32543 from ulysses-you/SPARK-35332-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-14 17:10:21 +08:00
Pablo Langa 9ea55fe771 [SPARK-35207][SQL] Normalize hash function behavior with negative zero (floating point types)
### What changes were proposed in this pull request?

Generally, we would expect that x = y => hash( x ) = hash( y ). However +-0 hash to different values for floating point types.
```
scala> spark.sql("select hash(cast('0.0' as double)), hash(cast('-0.0' as double))").show
+-------------------------+--------------------------+
|hash(CAST(0.0 AS DOUBLE))|hash(CAST(-0.0 AS DOUBLE))|
+-------------------------+--------------------------+
|              -1670924195|                -853646085|
+-------------------------+--------------------------+
scala> spark.sql("select cast('0.0' as double) == cast('-0.0' as double)").show
+--------------------------------------------+
|(CAST(0.0 AS DOUBLE) = CAST(-0.0 AS DOUBLE))|
+--------------------------------------------+
|                                        true|
+--------------------------------------------+
```
Here is an extract from IEEE 754:

> The two zeros are distinguishable arithmetically only by either division-byzero ( producing appropriately signed infinities ) or else by the CopySign function recommended by IEEE 754 /854. Infinities, SNaNs, NaNs and Subnormal numbers necessitate four more special cases

From this, I deduce that the hash function must produce the same result for 0 and -0.

### Why are the changes needed?

It is a correctness issue

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

This changes only affect to the hash function applied to -0 value in float and double types

### How was this patch tested?

Unit testing and manual testing

Closes #32496 from planga82/feature/spark35207_hashnegativezero.

Authored-by: Pablo Langa <soypab@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-14 12:40:36 +08:00
Takeshi Yamamuro 8fa739fb9d [SPARK-35329][SQL] Split generated switch code into pieces in ExpandExec
### What changes were proposed in this pull request?

This PR intends to split generated switch code into smaller ones in `ExpandExec`. In the current master, even a simple query like the one below generates a large method whose size (`maxMethodCodeSize:7448`) is close to `8000` (`CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT`);
```
scala> val df = Seq(("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id")
scala> val rdf = df.select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value").orderBy($"window.start".asc, $"value".desc).select("value")
scala> sql("SET spark.sql.adaptive.enabled=false")
scala> import org.apache.spark.sql.execution.debug._
scala> rdf.debugCodegen

Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 (maxMethodCodeSize:7448; maxConstantPoolSize:189(0.29% used); numInnerClasses:0) ==
                                    ^^^^
*(1) Project [window#34.start AS _gen_alias_39#39, value#11]
+- *(1) Filter ((isnotnull(window#34) AND (cast(time#10 as timestamp) >= window#34.start)) AND (cast(time#10 as timestamp) < window#34.end))
   +- *(1) Expand [List(named_struct(start, precisetimestampcon...

/* 028 */   private void expand_doConsume_0(InternalRow localtablescan_row_0, UTF8String expand_expr_0_0, boolean expand_exprIsNull_0_0, int expand_expr_1_0) throws java.io.IOException {
/* 029 */     boolean expand_isNull_0 = true;
/* 030 */     InternalRow expand_value_0 =
/* 031 */     null;
/* 032 */     for (int expand_i_0 = 0; expand_i_0 < 4; expand_i_0 ++) {
/* 033 */       switch (expand_i_0) {
/* 034 */       case 0:
                  (too many code lines)
/* 517 */         break;
/* 518 */
/* 519 */       case 1:
                  (too many code lines)
/* 1002 */         break;
/* 1003 */
/* 1004 */       case 2:
                  (too many code lines)
/* 1487 */         break;
/* 1488 */
/* 1489 */       case 3:
                  (too many code lines)
/* 1972 */         break;
/* 1973 */       }
/* 1974 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[33] /* numOutputRows */).add(1);
/* 1975 */
/* 1976 */       do {
/* 1977 */         boolean filter_value_2 = !expand_isNull_0;
/* 1978 */         if (!filter_value_2) continue;
```
The fix in this PR can make the method smaller as follows;
```
Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 (maxMethodCodeSize:1713; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
                                    ^^^^
*(1) Project [window#17.start AS _gen_alias_32#32, value#11]
+- *(1) Filter ((isnotnull(window#17) AND (cast(time#10 as timestamp) >= window#17.start)) AND (cast(time#10 as timestamp) < window#17.end))
   +- *(1) Expand [List(named_struct(start, precisetimestampcon...

/* 032 */   private void expand_doConsume_0(InternalRow localtablescan_row_0, UTF8String expand_expr_0_0, boolean expand_exprIsNull_0_0, int expand_expr_1_0) throws java.io.IOException {
/* 033 */     for (int expand_i_0 = 0; expand_i_0 < 4; expand_i_0 ++) {
/* 034 */       switch (expand_i_0) {
/* 035 */       case 0:
/* 036 */         expand_switchCaseCode_0(expand_exprIsNull_0_0, expand_expr_0_0);
/* 037 */         break;
/* 038 */
/* 039 */       case 1:
/* 040 */         expand_switchCaseCode_1(expand_exprIsNull_0_0, expand_expr_0_0);
/* 041 */         break;
/* 042 */
/* 043 */       case 2:
/* 044 */         expand_switchCaseCode_2(expand_exprIsNull_0_0, expand_expr_0_0);
/* 045 */         break;
/* 046 */
/* 047 */       case 3:
/* 048 */         expand_switchCaseCode_3(expand_exprIsNull_0_0, expand_expr_0_0);
/* 049 */         break;
/* 050 */       }
/* 051 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[33] /* numOutputRows */).add(1);
/* 052 */
/* 053 */       do {
/* 054 */         boolean filter_value_2 = !expand_resultIsNull_0;
/* 055 */         if (!filter_value_2) continue;
/* 056 */
...
```

### Why are the changes needed?

For better generated code.

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

No.

### How was this patch tested?

GA passed.

Closes #32457 from maropu/splitSwitchCode.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-13 17:53:46 -07:00
Liang-Chi Hsieh 6a949d1659 [SPARK-35397][SQL] Replace sys.err usage with explicit exception type
### What changes were proposed in this pull request?

This patch replaces `sys.err` usages with explicit exception types.

### Why are the changes needed?

Motivated by the previous comment https://github.com/apache/spark/pull/32519#discussion_r630787080, it sounds better to replace `sys.err` usages with explicit exception type.

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

No.

### How was this patch tested?

Existing tests.

Closes #32535 from viirya/replace-sys-err.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-13 10:37:24 -07:00
Linhong Liu 6aa2594c6b [SPARK-35366][SQL] Avoid using deprecated buildForBatch and buildForStreaming
### What changes were proposed in this pull request?
Currently, in DSv2, we are still using the deprecated `buildForBatch` and `buildForStreaming`.
This PR implements the `build`, `toBatch`, `toStreaming` interfaces to replace the deprecated ones.

### Why are the changes needed?
Code refactor

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

### How was this patch tested?
exsting UT

Closes #32497 from linhongliu-db/dsv2-writer.

Lead-authored-by: Linhong Liu <linhong.liu@databricks.com>
Co-authored-by: Linhong Liu <67896261+linhongliu-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 17:23:08 +00:00
gengjiaan c2e15cccab [SPARK-35062][SQL] Group exception messages in sql/streaming
### What changes were proposed in this pull request?
This PR group exception messages in `sql/core/src/main/scala/org/apache/spark/sql/streaming`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32464 from beliefer/SPARK-35062.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 15:04:03 +00:00
ulysses-you 6f63057ede [SPARK-35332][SQL] Make cache plan disable configs configurable
### What changes were proposed in this pull request?

Add a new config to make cache plan disable configs configurable.

### Why are the changes needed?

The disable configs of cache plan if to avoid the perfermance regression, but not all the query will slow than before due to AQE or bucket scan enabled. It's useful to make a new config so that user can decide if some configs should be disabled during cache plan.

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

Yes, a new config.

### How was this patch tested?

Add test.

Closes #32482 from ulysses-you/SPARK-35332.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 14:49:05 +00:00
Gengliang Wang 02c99f15ee [SPARK-35162][SQL] New SQL functions: TRY_ADD/TRY_DIVIDE
### What changes were proposed in this pull request?

Add New SQL functions:
* TRY_ADD
* TRY_DIVIDE

These expressions are identical to the following expression under ANSI mode except that it returns null if error occurs:
* ADD
* DIVIDE

Note: it is easy to add other expressions like `TRY_SUBTRACT`/`TRY_MULTIPLY` but let's control the number of these new expressions and just add `TRY_ADD` and `TRY_DIVIDE` for now.

### Why are the changes needed?

1. Users can manage to finish queries without interruptions in ANSI mode.
2. Users can get NULLs instead of unreasonable results if overflow occurs when ANSI mode is off.
For example, the behavior of the following SQL operations is unreasonable:
```
2147483647 + 2 => -2147483647
```

With the new safe version SQL functions:
```
TRY_ADD(2147483647, 2) => null
```

Note: **We should only add new expressions to important operators, instead of adding new safe expressions for all the expressions that can throw errors.**
### Does this PR introduce _any_ user-facing change?

Yes, new SQL functions: TRY_ADD/TRY_DIVIDE

### How was this patch tested?

Unit test

Closes #32292 from gengliangwang/try_add.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-13 22:26:08 +08:00
jiake b6d57b6b99 [SPARK-34637][SQL] Support DPP + AQE when the broadcast exchange can be reused
### What changes were proposed in this pull request?
We have supported DPP in AQE when the join is Broadcast hash join before applying the AQE rules in [SPARK-34168](https://issues.apache.org/jira/browse/SPARK-34168), which has some limitations. It only apply DPP when the small table side executed firstly and then the big table side can reuse the broadcast exchange in small table side. This PR is to address the above limitations and can apply the DPP when the broadcast exchange can be reused.

### Why are the changes needed?
Resolve the limitations when both enabling DPP and AQE

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

### How was this patch tested?
Adding new ut

Closes #31756 from JkSelf/supportDPP2.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 13:07:02 +00:00
Wenchen Fan d1b8bd7d11 [SPARK-34720][SQL] MERGE ... UPDATE/INSERT * should do by-name resolution
### What changes were proposed in this pull request?

In Spark, we have an extension in the MERGE syntax: INSERT/UPDATE *. This is not from ANSI standard or any other mainstream databases, so we need to define the behaviors by our own.

The behavior today is very weird: assume the source table has `n1` columns, target table has `n2` columns. We generate the assignments by taking the first `min(n1, n2)` columns from source & target tables and pairing them by ordinal.

This PR proposes a more reasonable behavior: take all the columns from target table as keys, and find the corresponding columns from source table by name as values.

### Why are the changes needed?

Fix the MEREG INSERT/UPDATE * to be more user-friendly and easy to do schema evolution.

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

Yes, but MERGE is only supported by very few data sources.

### How was this patch tested?

new tests

Closes #32192 from cloud-fan/merge.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 12:58:24 +00:00
Cheng Su c1e995ac95 [SPARK-35350][SQL] Add code-gen for left semi sort merge join
### What changes were proposed in this pull request?

As title. This PR is to add code-gen support for LEFT SEMI sort merge join. The main change is to add `semiJoin` code path in `SortMergeJoinExec.doProduce()` and introduce `onlyBufferFirstMatchedRow` in `SortMergeJoinExec.genScanner()`. The latter is for left semi sort merge join without condition. For this kind of query, we don't need to buffer all matched rows, but only the first one (this is same as non-code-gen code path).

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"))
val df2 = spark.range(4).select($"id".as("k2"))
val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi")
```

Example of generated code for the query:

```
== Subtree 5 / 5 (maxMethodCodeSize:302; maxConstantPoolSize:156(0.24% used); numInnerClasses:0) ==
*(5) Project [id#0L AS k1#2L]
+- *(5) SortMergeJoin [id#0L], [k2#6L], LeftSemi
   :- *(2) Sort [id#0L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#0L, 5), ENSURE_REQUIREMENTS, [id=#27]
   :     +- *(1) Range (0, 10, step=1, splits=2)
   +- *(4) Sort [k2#6L ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(k2#6L, 5), ENSURE_REQUIREMENTS, [id=#33]
         +- *(3) Project [id#4L AS k2#6L]
            +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(1, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     smj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */
/* 032 */   }
/* 033 */
/* 034 */   private boolean findNextJoinRows(
/* 035 */     scala.collection.Iterator streamedIter,
/* 036 */     scala.collection.Iterator bufferedIter) {
/* 037 */     smj_streamedRow_0 = null;
/* 038 */     int comp = 0;
/* 039 */     while (smj_streamedRow_0 == null) {
/* 040 */       if (!streamedIter.hasNext()) return false;
/* 041 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 042 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 043 */       if (false) {
/* 044 */         smj_streamedRow_0 = null;
/* 045 */         continue;
/* 046 */
/* 047 */       }
/* 048 */       if (!smj_matches_0.isEmpty()) {
/* 049 */         comp = 0;
/* 050 */         if (comp == 0) {
/* 051 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 052 */         }
/* 053 */
/* 054 */         if (comp == 0) {
/* 055 */           return true;
/* 056 */         }
/* 057 */         smj_matches_0.clear();
/* 058 */       }
/* 059 */
/* 060 */       do {
/* 061 */         if (smj_bufferedRow_0 == null) {
/* 062 */           if (!bufferedIter.hasNext()) {
/* 063 */             smj_value_3 = smj_value_0;
/* 064 */             return !smj_matches_0.isEmpty();
/* 065 */           }
/* 066 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 067 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 068 */           if (false) {
/* 069 */             smj_bufferedRow_0 = null;
/* 070 */             continue;
/* 071 */           }
/* 072 */           smj_value_2 = smj_value_1;
/* 073 */         }
/* 074 */
/* 075 */         comp = 0;
/* 076 */         if (comp == 0) {
/* 077 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 078 */         }
/* 079 */
/* 080 */         if (comp > 0) {
/* 081 */           smj_bufferedRow_0 = null;
/* 082 */         } else if (comp < 0) {
/* 083 */           if (!smj_matches_0.isEmpty()) {
/* 084 */             smj_value_3 = smj_value_0;
/* 085 */             return true;
/* 086 */           } else {
/* 087 */             smj_streamedRow_0 = null;
/* 088 */           }
/* 089 */         } else {
/* 090 */           if (smj_matches_0.isEmpty()) {
/* 091 */             smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 092 */           }
/* 093 */
/* 094 */           smj_bufferedRow_0 = null;
/* 095 */         }
/* 096 */       } while (smj_streamedRow_0 != null);
/* 097 */     }
/* 098 */     return false; // unreachable
/* 099 */   }
/* 100 */
/* 101 */   protected void processNext() throws java.io.IOException {
/* 102 */     while (findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0)) {
/* 103 */       long smj_value_4 = -1L;
/* 104 */       smj_value_4 = smj_streamedRow_0.getLong(0);
/* 105 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 106 */       boolean smj_hasOutputRow_0 = false;
/* 107 */
/* 108 */       while (!smj_hasOutputRow_0 && smj_iterator_0.hasNext()) {
/* 109 */         InternalRow smj_bufferedRow_1 = (InternalRow) smj_iterator_0.next();
/* 110 */
/* 111 */         smj_hasOutputRow_0 = true;
/* 112 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 113 */
/* 114 */         // common sub-expressions
/* 115 */
/* 116 */         smj_mutableStateArray_0[1].reset();
/* 117 */
/* 118 */         smj_mutableStateArray_0[1].write(0, smj_value_4);
/* 119 */         append((smj_mutableStateArray_0[1].getRow()).copy());
/* 120 */
/* 121 */       }
/* 122 */       if (shouldStop()) return;
/* 123 */     }
/* 124 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 125 */   }
/* 126 */
/* 127 */ }
```

### Why are the changes needed?

Improve query CPU performance. Test with one query:

```
 def sortMergeJoin(): Unit = {
    val N = 2 << 20
    codegenBenchmark("left semi sort merge join", N) {
      val df1 = spark.range(N).selectExpr(s"id * 2 as k1")
      val df2 = spark.range(N).selectExpr(s"id * 3 as k2")
      val df = df1.join(df2, col("k1") === col("k2"), "left_semi")
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
      df.noop()
    }
  }
```

Seeing 30% of run-time improvement:

```
Running benchmark: left semi sort merge join
  Running case: left semi sort merge join code-gen off
  Stopped after 2 iterations, 1369 ms
  Running case: left semi sort merge join code-gen on
  Stopped after 5 iterations, 2743 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi sort merge join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
left semi sort merge join code-gen off              676            685          13          3.1         322.2       1.0X
left semi sort merge join code-gen on               524            549          32          4.0         249.7       1.3X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala` and `ExistenceJoinSuite.scala`.

Closes #32528 from c21/smj-left-semi.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-13 12:52:26 +00:00
Kent Yao 51815430b2 [SPARK-35380][SQL] Loading SparkSessionExtensions from ServiceLoader
### What changes were proposed in this pull request?

In https://github.com/yaooqinn/itachi/issues/8, we had a discussion about the current extension injection for the spark session.  We've agreed that the current way is not that convenient for both third-party developers and end-users.

It's much simple if third-party developers can provide a resource file that contains default extensions for Spark to  load ahead

### Why are the changes needed?

better use experience

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

no, dev change

### How was this patch tested?

new tests

Closes #32515 from yaooqinn/SPARK-35380.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-13 16:34:13 +08:00
Chao Sun 0ab9bd79b3 [SPARK-35384][SQL] Improve performance for InvokeLike.invoke
### What changes were proposed in this pull request?

Change `map` in `InvokeLike.invoke` to a while loop to improve performance, following Spark [style guide](https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex).

### Why are the changes needed?

`InvokeLike.invoke`, which is used in non-codegen path for `Invoke` and `StaticInvoke`, currently uses `map` to evaluate arguments:
```scala
val args = arguments.map(e => e.eval(input).asInstanceOf[Object])
if (needNullCheck && args.exists(_ == null)) {
  // return null if one of arguments is null
  null
} else {
  ...
```
which is pretty expensive if the method itself is trivial. We can change it to a plain while loop.

<img width="871" alt="Screen Shot 2021-05-12 at 12 19 59 AM" src="https://user-images.githubusercontent.com/506679/118055719-7f985a00-b33d-11eb-943b-cf85eab35f44.png">

Benchmark results show this can improve as much as 3x from `V2FunctionBenchmark`:

Before
```
 OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Linux 5.4.0-1046-azure
 Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
 scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 --------------------------------------------------------------------------------------------------------------------------------------------------------------
 native_long_add                                                                         36506          36656         251         13.7          73.0       1.0X
 java_long_add_default                                                                   47151          47540         370         10.6          94.3       0.8X
 java_long_add_magic                                                                    178691         182457        1327          2.8         357.4       0.2X
 java_long_add_static_magic                                                             177151         178258        1151          2.8         354.3       0.2X
```

After
```
 OpenJDK 64-Bit Server VM 1.8.0_292-b10 on Linux 5.4.0-1046-azure
 Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
 scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 --------------------------------------------------------------------------------------------------------------------------------------------------------------
 native_long_add                                                                         29897          30342         568         16.7          59.8       1.0X
 java_long_add_default                                                                   40628          41075         664         12.3          81.3       0.7X
 java_long_add_magic                                                                     54553          54755         182          9.2         109.1       0.5X
 java_long_add_static_magic                                                              55410          55532         127          9.0         110.8       0.5X
```

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

No

### How was this patch tested?

Existing tests.

Closes #32527 from sunchao/SPARK-35384.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-12 20:57:21 -07:00
Takeshi Yamamuro 3241aeb7f4 [SPARK-35385][SQL][TESTS] Skip duplicate queries in the TPCDS-related tests
### What changes were proposed in this pull request?

This PR proposes to skip the "q6", "q34", "q64", "q74", "q75", "q78" queries in the TPCDS-related tests because the TPCDS v2.7 queries have almost the same ones; the only differences in these queries are ORDER BY columns.

### Why are the changes needed?

To improve test performance.

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

No, dev only.

### How was this patch tested?

Existing tests.

Closes #32520 from maropu/SkipDupQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-13 09:46:25 +09:00
Chao Sun bc95c3a69b [SPARK-35361][SQL][FOLLOWUP] Switch to use while loop
### What changes were proposed in this pull request?

Switch to plain `while` loop following Spark [style guide](https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex).

### Why are the changes needed?

`while` loop may yield better performance comparing to `foreach`.

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

No

### How was this patch tested?

N/A

Closes #32522 from sunchao/SPARK-35361-follow-up.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:41:12 -07:00
Liang-Chi Hsieh f156a95641 [SPARK-35347][SQL][FOLLOWUP] Throw exception with an explicit exception type when cannot find the method instead of sys.error
### What changes were proposed in this pull request?

A simple follow-up of #32474 to throw exception instead of sys.error.

### Why are the changes needed?

An exception only fails the query, instead of sys.error.

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

Yes, if `Invoke` or `StaticInvoke` cannot find the method, instead of original `sys.error` now we only throw an exception.

### How was this patch tested?

Existing tests.

Closes #32519 from viirya/SPARK-35347-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-12 09:56:08 -07:00
Cheng Su 7bcadedbd2 [SPARK-35349][SQL] Add code-gen for left/right outer sort merge join
### What changes were proposed in this pull request?

This PR is to add code-gen support for LEFT OUTER / RIGHT OUTER sort merge join. Currently sort merge join only supports inner join type (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala#L374 ). There's no fundamental reason why we cannot support code-gen for other join types. Here we add code-gen for LEFT OUTER / RIGHT OUTER join. Will submit followup PRs to add LEFT SEMI, LEFT ANTI and FULL OUTER code-gen separately.

The change is to extend current sort merge join logic to work with LEFT OUTER and RIGHT OUTER (should work with LEFT SEMI/ANTI as well, but FULL OUTER join needs some other more code change). Replace left/right with streamed/buffered to make code extendable to other join types besides inner join.

Example query:

```
val df1 = spark.range(10).select($"id".as("k1"), $"id".as("k3"))
val df2 = spark.range(4).select($"id".as("k2"), $"id".as("k4"))
df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2" && $"k3" + 1 < $"k4", "left_outer").explain("codegen")
```

Example generated code:

```
== Subtree 5 / 5 (maxMethodCodeSize:396; maxConstantPoolSize:159(0.24% used); numInnerClasses:0) ==
*(5) SortMergeJoin [k1#2L], [k2#8L], LeftOuter, ((k3#3L + 1) < k4#9L)
:- *(2) Sort [k1#2L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(k1#2L, 5), ENSURE_REQUIREMENTS, [id=#26]
:     +- *(1) Project [id#0L AS k1#2L, id#0L AS k3#3L]
:        +- *(1) Range (0, 10, step=1, splits=2)
+- *(4) Sort [k2#8L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k2#8L, 5), ENSURE_REQUIREMENTS, [id=#32]
      +- *(3) Project [id#6L AS k2#8L, id#6L AS k4#9L]
         +- *(3) Range (0, 4, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage5(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=5
/* 006 */ final class GeneratedIteratorForCodegenStage5 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator smj_streamedInput_0;
/* 010 */   private scala.collection.Iterator smj_bufferedInput_0;
/* 011 */   private InternalRow smj_streamedRow_0;
/* 012 */   private InternalRow smj_bufferedRow_0;
/* 013 */   private long smj_value_2;
/* 014 */   private org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray smj_matches_0;
/* 015 */   private long smj_value_3;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] smj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage5(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     smj_streamedInput_0 = inputs[0];
/* 026 */     smj_bufferedInput_0 = inputs[1];
/* 027 */
/* 028 */     smj_matches_0 = new org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray(2147483632, 2147483647);
/* 029 */     smj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(4, 0);
/* 030 */
/* 031 */   }
/* 032 */
/* 033 */   private boolean findNextJoinRows(
/* 034 */     scala.collection.Iterator streamedIter,
/* 035 */     scala.collection.Iterator bufferedIter) {
/* 036 */     smj_streamedRow_0 = null;
/* 037 */     int comp = 0;
/* 038 */     while (smj_streamedRow_0 == null) {
/* 039 */       if (!streamedIter.hasNext()) return false;
/* 040 */       smj_streamedRow_0 = (InternalRow) streamedIter.next();
/* 041 */       long smj_value_0 = smj_streamedRow_0.getLong(0);
/* 042 */       if (false) {
/* 043 */         if (!smj_matches_0.isEmpty()) {
/* 044 */           smj_matches_0.clear();
/* 045 */         }
/* 046 */         return false;
/* 047 */
/* 048 */       }
/* 049 */       if (!smj_matches_0.isEmpty()) {
/* 050 */         comp = 0;
/* 051 */         if (comp == 0) {
/* 052 */           comp = (smj_value_0 > smj_value_3 ? 1 : smj_value_0 < smj_value_3 ? -1 : 0);
/* 053 */         }
/* 054 */
/* 055 */         if (comp == 0) {
/* 056 */           return true;
/* 057 */         }
/* 058 */         smj_matches_0.clear();
/* 059 */       }
/* 060 */
/* 061 */       do {
/* 062 */         if (smj_bufferedRow_0 == null) {
/* 063 */           if (!bufferedIter.hasNext()) {
/* 064 */             smj_value_3 = smj_value_0;
/* 065 */             return !smj_matches_0.isEmpty();
/* 066 */           }
/* 067 */           smj_bufferedRow_0 = (InternalRow) bufferedIter.next();
/* 068 */           long smj_value_1 = smj_bufferedRow_0.getLong(0);
/* 069 */           if (false) {
/* 070 */             smj_bufferedRow_0 = null;
/* 071 */             continue;
/* 072 */           }
/* 073 */           smj_value_2 = smj_value_1;
/* 074 */         }
/* 075 */
/* 076 */         comp = 0;
/* 077 */         if (comp == 0) {
/* 078 */           comp = (smj_value_0 > smj_value_2 ? 1 : smj_value_0 < smj_value_2 ? -1 : 0);
/* 079 */         }
/* 080 */
/* 081 */         if (comp > 0) {
/* 082 */           smj_bufferedRow_0 = null;
/* 083 */         } else if (comp < 0) {
/* 084 */           if (!smj_matches_0.isEmpty()) {
/* 085 */             smj_value_3 = smj_value_0;
/* 086 */             return true;
/* 087 */           } else {
/* 088 */             return false;
/* 089 */           }
/* 090 */         } else {
/* 091 */           smj_matches_0.add((UnsafeRow) smj_bufferedRow_0);
/* 092 */           smj_bufferedRow_0 = null;
/* 093 */         }
/* 094 */       } while (smj_streamedRow_0 != null);
/* 095 */     }
/* 096 */     return false; // unreachable
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     while (smj_streamedInput_0.hasNext()) {
/* 101 */       findNextJoinRows(smj_streamedInput_0, smj_bufferedInput_0);
/* 102 */       long smj_value_4 = -1L;
/* 103 */       long smj_value_5 = -1L;
/* 104 */       boolean smj_loaded_0 = false;
/* 105 */       smj_value_5 = smj_streamedRow_0.getLong(1);
/* 106 */       scala.collection.Iterator<UnsafeRow> smj_iterator_0 = smj_matches_0.generateIterator();
/* 107 */       boolean smj_foundMatch_0 = false;
/* 108 */
/* 109 */       // the last iteration of this loop is to emit an empty row if there is no matched rows.
/* 110 */       while (smj_iterator_0.hasNext() || !smj_foundMatch_0) {
/* 111 */         InternalRow smj_bufferedRow_1 = smj_iterator_0.hasNext() ?
/* 112 */         (InternalRow) smj_iterator_0.next() : null;
/* 113 */         boolean smj_isNull_5 = true;
/* 114 */         long smj_value_9 = -1L;
/* 115 */         if (smj_bufferedRow_1 != null) {
/* 116 */           long smj_value_8 = smj_bufferedRow_1.getLong(1);
/* 117 */           smj_isNull_5 = false;
/* 118 */           smj_value_9 = smj_value_8;
/* 119 */         }
/* 120 */         if (smj_bufferedRow_1 != null) {
/* 121 */           boolean smj_isNull_6 = true;
/* 122 */           boolean smj_value_10 = false;
/* 123 */           long smj_value_11 = -1L;
/* 124 */
/* 125 */           smj_value_11 = smj_value_5 + 1L;
/* 126 */
/* 127 */           if (!smj_isNull_5) {
/* 128 */             smj_isNull_6 = false; // resultCode could change nullability.
/* 129 */             smj_value_10 = smj_value_11 < smj_value_9;
/* 130 */
/* 131 */           }
/* 132 */           if (smj_isNull_6 || !smj_value_10) {
/* 133 */             continue;
/* 134 */           }
/* 135 */         }
/* 136 */         if (!smj_loaded_0) {
/* 137 */           smj_loaded_0 = true;
/* 138 */           smj_value_4 = smj_streamedRow_0.getLong(0);
/* 139 */         }
/* 140 */         boolean smj_isNull_3 = true;
/* 141 */         long smj_value_7 = -1L;
/* 142 */         if (smj_bufferedRow_1 != null) {
/* 143 */           long smj_value_6 = smj_bufferedRow_1.getLong(0);
/* 144 */           smj_isNull_3 = false;
/* 145 */           smj_value_7 = smj_value_6;
/* 146 */         }
/* 147 */         smj_foundMatch_0 = true;
/* 148 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 149 */
/* 150 */         smj_mutableStateArray_0[0].reset();
/* 151 */
/* 152 */         smj_mutableStateArray_0[0].zeroOutNullBytes();
/* 153 */
/* 154 */         smj_mutableStateArray_0[0].write(0, smj_value_4);
/* 155 */
/* 156 */         smj_mutableStateArray_0[0].write(1, smj_value_5);
/* 157 */
/* 158 */         if (smj_isNull_3) {
/* 159 */           smj_mutableStateArray_0[0].setNullAt(2);
/* 160 */         } else {
/* 161 */           smj_mutableStateArray_0[0].write(2, smj_value_7);
/* 162 */         }
/* 163 */
/* 164 */         if (smj_isNull_5) {
/* 165 */           smj_mutableStateArray_0[0].setNullAt(3);
/* 166 */         } else {
/* 167 */           smj_mutableStateArray_0[0].write(3, smj_value_9);
/* 168 */         }
/* 169 */         append((smj_mutableStateArray_0[0].getRow()).copy());
/* 170 */
/* 171 */       }
/* 172 */       if (shouldStop()) return;
/* 173 */     }
/* 174 */     ((org.apache.spark.sql.execution.joins.SortMergeJoinExec) references[1] /* plan */).cleanupResources();
/* 175 */   }
/* 176 */
/* 177 */ }
```

### Why are the changes needed?

Improve query CPU performance. Example micro benchmark below showed 10% run-time improvement.

```
def sortMergeJoinWithDuplicates(): Unit = {
    val N = 2 << 20
    codegenBenchmark("sort merge join with duplicates", N) {
      val df1 = spark.range(N)
        .selectExpr(s"(id * 15485863) % ${N*10} as k1", "id as k3")
      val df2 = spark.range(N)
        .selectExpr(s"(id * 15485867) % ${N*10} as k2", "id as k4")
      val df = df1.join(df2, col("k1") === col("k2") && col("k3") * 3 < col("k4"), "left_outer")
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined)
      df.noop()
    }
 }
```

```
Running benchmark: sort merge join with duplicates
  Running case: sort merge join with duplicates outer-smj-codegen off
  Stopped after 2 iterations, 2696 ms
  Running case: sort merge join with duplicates outer-smj-codegen on
  Stopped after 5 iterations, 6058 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.16
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
sort merge join with duplicates:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------------------
sort merge join with duplicates outer-smj-codegen off           1333           1348          21          1.6         635.7       1.0X
sort merge join with duplicates outer-smj-codegen on            1169           1212          47          1.8         557.4       1.1X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite.scala` and `WholeStageCodegenSuite.scala`.

Closes #32476 from c21/smj-outer-codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 14:10:15 +00:00
Takeshi Yamamuro 101b0cc313 [SPARK-35253][SQL][BUILD] Bump up the janino version to v3.1.4
### What changes were proposed in this pull request?

This PR proposes to bump up the janino version from 3.0.16 to v3.1.4.
The major changes of this upgrade are as follows:
 - Fixed issue #131: Janino 3.1.2 is 10x slower than 3.0.11: The Compiler's IClassLoader was initialized way too eagerly, thus lots of classes were loaded from the class path, which is very slow.
 - Improved the encoding of stack map frames according to JVMS11 4.7.4: Previously, only "full_frame"s were generated.
 - Fixed issue #107: Janino requires "org.codehaus.commons.compiler.io", but commons-compiler does not export this package
 - Fixed the promotion of the array access index expression (see JLS7 15.13 Array Access Expressions).

For all the changes, please see the change log: http://janino-compiler.github.io/janino/changelog.html

NOTE1: I've checked that there is no obvious performance regression. For all the data, see a link: https://docs.google.com/spreadsheets/d/1srxT9CioGQg1fLKM3Uo8z1sTzgCsMj4pg6JzpdcG6VU/edit?usp=sharing

NOTE2: We upgraded janino to 3.1.2 (#27860) once before, but the commit had been reverted in #29495 because of the correctness issue. Recently, #32374 had checked if Spark could land on v3.1.3 or not, but a new bug was found there. These known issues has been fixed in v3.1.4 by following PRs:
 - janino-compiler/janino#145
 - janino-compiler/janino#146

### Why are the changes needed?

janino v3.0.X  is no longer maintained.

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

No.

### How was this patch tested?

GA passed.

Closes #32455 from maropu/janino_v3.1.4.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-12 08:57:57 -05:00
Angerszhuuuu ed059541eb [SPARK-29145][SQL][FOLLOWUP] Clean up code about support sub-queries in join conditions
### What changes were proposed in this pull request?
According to discuss https://github.com/apache/spark/pull/25854#discussion_r629451135

### Why are the changes needed?
Clean code

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

### How was this patch tested?
Existed UT

Closes #32499 from AngersZhuuuu/SPARK-29145-fix.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 13:45:53 +00:00
Yingyi Bu d92018ee35 [SPARK-35298][SQL] Migrate to transformWithPruning for rules in Optimizer.scala
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- ALIAS
- AND_OR
- AVERAGE
- GENERATE
- INTERSECT
- SORT
- SUM
- DISTINCT_LIKE
- PROJECT
- REPARTITION_OPERATION
- UNION

Added tree traversal pruning to the following rules in Optimizer.scala:
- EliminateAggregateFilter
- RemoveRedundantAggregates
- RemoveNoopOperators
- RemoveNoopUnion
- LimitPushDown
- ColumnPruning
- CollapseRepartition
- OptimizeRepartition
- OptimizeWindowFunctions
- CollapseWindow
- TransposeWindow
- InferFiltersFromGenerate
- InferFiltersFromConstraints
- CombineUnions
- CombineFilters
- EliminateSorts
- PruneFilters
- EliminateLimits
- DecimalAggregates
- ConvertToLocalRelation
- ReplaceDistinctWithAggregate
- ReplaceIntersectWithSemiJoin
- ReplaceExceptWithAntiJoin
- RewriteExceptAll
- RewriteIntersectAll
- RemoveLiteralFromGroupExpressions
- RemoveRepetitionFromGroupExpressions
- OptimizeLimitZero

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

perf diff:
Rule name | Total Time (baseline) | Total Time (experiment) | experiment/baseline
RemoveRedundantAggregates | 51290766 | 67070477 | 1.31
RemoveNoopOperators | 192371141 | 196631275 | 1.02
RemoveNoopUnion | 49222561 | 43266681 | 0.88
LimitPushDown | 40885185 | 21672646 | 0.53
ColumnPruning | 2003406120 | 1285562149 | 0.64
CollapseRepartition | 40648048 | 72646515 | 1.79
OptimizeRepartition | 37813850 | 20600803 | 0.54
OptimizeWindowFunctions | 174426904 | 46741409 | 0.27
CollapseWindow | 38959957 | 24542426 | 0.63
TransposeWindow | 33533191 | 20414930 | 0.61
InferFiltersFromGenerate | 21758688 | 15597344 | 0.72
InferFiltersFromConstraints | 518009794 | 493282321 | 0.95
CombineUnions | 67694022 | 70550382 | 1.04
CombineFilters | 35265060 | 29005424 | 0.82
EliminateSorts | 57025509 | 19795776 | 0.35
PruneFilters | 433964815 | 465579200 | 1.07
EliminateLimits | 44275393 | 24476859 | 0.55
DecimalAggregates | 83143172 | 28816090 | 0.35
ReplaceDistinctWithAggregate | 21783760 | 18287489 | 0.84
ReplaceIntersectWithSemiJoin | 22311271 | 16566393 | 0.74
ReplaceExceptWithAntiJoin | 23838520 | 16588808 | 0.70
RewriteExceptAll | 32750296 | 29421957 | 0.90
RewriteIntersectAll | 29760454 | 21243599 | 0.71
RemoveLiteralFromGroupExpressions | 28151861 | 25270947 | 0.90
RemoveRepetitionFromGroupExpressions | 29587030 | 23447041 | 0.79
OptimizeLimitZero | 18081943 | 15597344 | 0.86
**Accumulated | 4129959311 | 3112676285 | 0.75**

### How was this patch tested?

Existing tests.

Closes #32439 from sigmod/optimizer.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-12 20:42:47 +08:00
PengLei 82c520a3e2 [SPARK-35243][SQL] Support columnar execution on ANSI interval types
### What changes were proposed in this pull request?
Columnar execution support for ANSI interval types include YearMonthIntervalType and DayTimeIntervalType

### Why are the changes needed?
support cache tables with ANSI interval types.

### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
run ./dev/lint-java
run ./dev/scalastyle
run test: CachedTableSuite
run test: ColumnTypeSuite

Closes #32452 from Peng-Lei/SPARK-35243.

Lead-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: Lei Peng <peng.8lei@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 20:11:34 +09:00
Chao Sun 78221bda95 [SPARK-35361][SQL] Improve performance for ApplyFunctionExpression
### What changes were proposed in this pull request?

In `ApplyFunctionExpression`, move `zipWithIndex` out of the loop for each input row.

### Why are the changes needed?

When the `ScalarFunction` is trivial, `zipWithIndex` could incur significant costs, as shown below:

<img width="899" alt="Screen Shot 2021-05-11 at 10 03 42 AM" src="https://user-images.githubusercontent.com/506679/117866421-fb19de80-b24b-11eb-8c94-d5e8c8b1eda9.png">

By removing it out of the loop, I'm seeing sometimes 2x speedup from `V2FunctionBenchmark`. For instance:

Before:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         32437          32896         434         15.4          64.9       1.0X
java_long_add_default                                                                   85675          97045         NaN          5.8         171.3       0.4X
```

After:
```
scalar function (long + long) -> long, result_nullable = false codegen = false:  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
native_long_add                                                                         30182          30387         279         16.6          60.4       1.0X
java_long_add_default                                                                   42862          43009         209         11.7          85.7       0.7X
```

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

No

### How was this patch tested?

Existing tests

Closes #32507 from sunchao/SPARK-35361.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:16:35 +09:00
Yingyi Bu 7c9a9ec04f [SPARK-35146][SQL] Migrate to transformWithPruning or resolveWithPruning for rules in finishAnalysis.scala
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- BOOL_AGG
- COUNT_IF
- CURRENT_LIKE
- RUNTIME_REPLACEABLE

Added tree traversal pruning to the following rules:
- ReplaceExpressions
- RewriteNonCorrelatedExists
- ComputeCurrentTime
- GetCurrentDatabaseAndCatalog

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

Performance improvement (org.apache.spark.sql.TPCDSQuerySuite):
Rule name | Total Time (baseline) | Total Time (experiment) | experiment/baseline
ReplaceExpressions | 27546369 | 19753804 | 0.72
RewriteNonCorrelatedExists | 17304883 | 2086194 | 0.12
ComputeCurrentTime | 35751301 | 19984477 | 0.56
GetCurrentDatabaseAndCatalog | 37230787 | 18874013 | 0.51

### How was this patch tested?

Existing tests.

Closes #32461 from sigmod/finish_analysis.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-11 17:11:38 +08:00
Cheng Su c4ca23207b [SPARK-35363][SQL] Refactor sort merge join code-gen be agnostic to join type
### What changes were proposed in this pull request?

This is a pre-requisite of https://github.com/apache/spark/pull/32476, in discussion of https://github.com/apache/spark/pull/32476#issuecomment-836469779 . This is to refactor sort merge join code-gen to depend on streamed/buffered terminology, which makes the code-gen agnostic to different join types and can be extended to support other join types than inner join.

### Why are the changes needed?

Pre-requisite of https://github.com/apache/spark/pull/32476.

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

No.

### How was this patch tested?

Existing unit test in `InnerJoinSuite.scala` for inner join code-gen.

Closes #32495 from c21/smj-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-11 11:21:59 +09:00
gengjiaan 44bd0a8bd3 [SPARK-35088][SQL][FOLLOWUP] Improve the error message for Sequence expression
### What changes were proposed in this pull request?
Sequence expression output a message looks confused.
This PR will fix the issue.

### Why are the changes needed?
Improve the error message for Sequence expression

### Does this PR introduce _any_ user-facing change?
Yes. this PR updates the error message of Sequence expression.

### How was this patch tested?
Tests updated.

Closes #32492 from beliefer/SPARK-35088-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-11 09:45:09 +09:00
Gengliang Wang d2a535f85b [SPARK-34246][FOLLOWUP] Change the definition of findTightestCommonType for backward compatibility
### What changes were proposed in this pull request?

Change the definition of `findTightestCommonType` from
```
def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType]
```
to
```
val findTightestCommonType: (DataType, DataType) => Option[DataType]
```

### Why are the changes needed?

For backward compatibility.
When running a MongoDB connector (built with Spark 3.1.1) with the latest master, there is such an error
```
java.lang.NoSuchMethodError: org.apache.spark.sql.catalyst.analysis.TypeCoercion$.findTightestCommonType()Lscala/Function2
```
from https://github.com/mongodb/mongo-spark/blob/master/src/main/scala/com/mongodb/spark/sql/MongoInferSchema.scala#L150

In the previous release, the function was
```
static public  scala.Function2<org.apache.spark.sql.types.DataType, org.apache.spark.sql.types.DataType, scala.Option<org.apache.spark.sql.types.DataType>> findTightestCommonType ()
```
After https://github.com/apache/spark/pull/31349, the function becomes:
```
static public  scala.Option<org.apache.spark.sql.types.DataType> findTightestCommonType (org.apache.spark.sql.types.DataType t1, org.apache.spark.sql.types.DataType t2)
```

This PR is to reduce the unnecessary API change.
### Does this PR introduce _any_ user-facing change?

Yes, the definition of `TypeCoercion.findTightestCommonType`  is consistent with previous release again.

### How was this patch tested?

Existing unit tests

Closes #32493 from gengliangwang/typecoercion.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-10 23:26:39 +08:00
Angerszhuuuu 7182f8cece [SPARK-35360][SQL] RepairTableCommand respects spark.sql.addPartitionInBatch.size too
### What changes were proposed in this pull request?
RepairTableCommand respects `spark.sql.addPartitionInBatch.size` too

### Why are the changes needed?
Make RepairTableCommand add partition batch size configurable.

### Does this PR introduce _any_ user-facing change?
User can use `spark.sql.addPartitionInBatch.size` to change batch size when repair table.

### How was this patch tested?
Not need

Closes #32489 from AngersZhuuuu/SPARK-35360.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-10 14:53:31 +05:00
Chao Sun 245dce1ea1 [SPARK-35261][SQL][TESTS][FOLLOW-UP] Change failOnError to false for NativeAdd in V2FunctionBenchmark
### What changes were proposed in this pull request?

Change `failOnError` to false for `NativeAdd` in `V2FunctionBenchmark`.

### Why are the changes needed?

Since `NativeAdd` is simply doing addition on long it's better to set `failOnError` to false so it will use native long addition instead of `Math.addExact`.

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

No

### How was this patch tested?

N/A

Closes #32481 from sunchao/SPARK-35261-follow-up.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-10 07:20:05 +00:00
Angerszhuuuu 2c8ced9590 [SPARK-35111][SPARK-35112][SQL][FOLLOWUP] Rename ANSI interval patterns and regexps
### What changes were proposed in this pull request?
Rename pattern strings and regexps of year-month and day-time intervals.

### Why are the changes needed?
To improve code maintainability.

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

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

Closes #32444 from AngersZhuuuu/SPARK-35111-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-10 11:33:27 +05:00
Cheng Su 38eb5a6936 [SPARK-35354][SQL] Replace BaseJoinExec with ShuffledJoin in CoalesceBucketsInJoin
### What changes were proposed in this pull request?

As title. We should use a more restrictive interface `ShuffledJoin` other than `BaseJoinExec` in `CoalesceBucketsInJoin`, as the rule only applies to sort merge join and shuffled hash join (i.e. `ShuffledJoin`).

### Why are the changes needed?

Code cleanup.

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

No.

### How was this patch tested?

Existing unit test in `CoalesceBucketsInJoinSuite`.

Closes #32480 from c21/minor-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-10 10:04:49 +09:00
Ruifeng Zheng 620f0727e3 [SPARK-35231][SQL] logical.Range override maxRowsPerPartition
### What changes were proposed in this pull request?
when `numSlices` is avaiable, `logical.Range` should compute a exact `maxRowsPerPartition`

### Why are the changes needed?
`maxRowsPerPartition` is used in optimizer, we should provide an exact value if possible

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

### How was this patch tested?
existing testsuites

Closes #32350 from zhengruifeng/range_maxRowsPerPartition.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-09 21:44:49 +09:00
Liang-Chi Hsieh 5b65d8a129 [SPARK-35347][SQL] Use MethodUtils for looking up methods in Invoke and StaticInvoke
### What changes were proposed in this pull request?

This patch proposes to use `MethodUtils` for looking up methods `Invoke` and `StaticInvoke` expressions.

### Why are the changes needed?

Currently we wrote our logic in `Invoke` and `StaticInvoke` expressions for looking up methods. It is tricky to consider all the cases and there is already existing utility package for this purpose. We should reuse the utility package.

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

No, internal change only.

### How was this patch tested?

Existing tests.

Closes #32474 from viirya/invoke-util.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 15:17:30 -07:00
Dongjoon Hyun e31bef1ed4 Revert "[SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client"
This reverts commit b4ec9e2304.
2021-05-08 13:01:17 -07:00
Takeshi Yamamuro 06c40091a6 [SPARK-35327][SQL][TESTS] Filters out the TPC-DS queries that can cause flaky test results
### What changes were proposed in this pull request?

This PR proposes to filter out TPCDS v1.4 q6 and q75 in `TPCDSQueryTestSuite`.

I saw`TPCDSQueryTestSuite` failed nondeterministically because output row orders were different with those in the golden files. For example, the failure in the GA job, https://github.com/linhongliu-db/spark/runs/2507928605?check_suite_focus=true, happened because the `tpcds/q6.sql` query output rows were only sorted by `cnt`:

a0c76a8755/sql/core/src/test/resources/tpcds/q6.sql (L20)
Actually, `tpcds/q6.sql`  and `tpcds-v2.7.0/q6.sql` are almost the same and the only difference is that `tpcds-v2.7.0/q6.sql` sorts both `cnt` and `a.ca_state`:
a0c76a8755/sql/core/src/test/resources/tpcds-v2.7.0/q6.sql (L22)
So, I think it's okay just to test `tpcds-v2.7.0/q6.sql` in this case (q75 has the same issue).

### Why are the changes needed?

For stable testing.

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

No, dev-only.

### How was this patch tested?

GA passed.

Closes #32454 from maropu/CleanUpTpcdsQueries.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-08 21:43:39 +09:00
Kent Yao b0257801d5 [SPARK-35331][SQL] Support resolving missing attrs for distribute/cluster by/repartition hint
### What changes were proposed in this pull request?

This PR makes the below case work well.

```sql
select a b from values(1) t(a) distribute by a;
```

```logtalk
== Parsed Logical Plan ==
'RepartitionByExpression ['a]
+- 'Project ['a AS b#42]
   +- 'SubqueryAlias t
      +- 'UnresolvedInlineTable [a], [List(1)]

== Analyzed Logical Plan ==
org.apache.spark.sql.AnalysisException: cannot resolve 'a' given input columns: [b]; line 1 pos 62;
'RepartitionByExpression ['a]
+- Project [a#48 AS b#42]
   +- SubqueryAlias t
      +- LocalRelation [a#48]
```
### Why are the changes needed?

bugfix

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

yes, the original attributes can be used in `distribute by` / `cluster by` and hints like `/*+ REPARTITION(3, c) */`

### How was this patch tested?

new tests

Closes #32465 from yaooqinn/SPARK-35331.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 05:00:51 -07:00
Chao Sun 323a6e848e [SPARK-35232][SQL] Nested column pruning should retain column metadata
### What changes were proposed in this pull request?

Retain column metadata during the process of nested column pruning, when constructing `StructField`.

To test the above change, this also added the logic of column projection in `InMemoryTable`. Without the fix `DSV2CharVarcharDDLTestSuite` will fail.

### Why are the changes needed?

The column metadata is used in a few places such as re-constructing CHAR/VARCHAR information such as in [SPARK-33901](https://issues.apache.org/jira/browse/SPARK-33901). Therefore, we should retain the info during nested column pruning.

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

No

### How was this patch tested?

Existing tests.

Closes #32354 from sunchao/SPARK-35232.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-07 22:37:54 -07:00
Chao Sun f47e0f8379 [SPARK-35261][SQL] Support static magic method for stateless Java ScalarFunction
### What changes were proposed in this pull request?

This allows `ScalarFunction` implemented in Java to optionally specify the magic method `invoke` to be static, which can be used if the UDF is stateless. Comparing to the non-static method, it can potentially give better performance due to elimination of dynamic dispatch, etc.

Also added a benchmark to measure performance of: the default `produceResult`, non-static magic method and static magic method.

### Why are the changes needed?

For UDFs that are stateless (e.g., no need to maintain intermediate state between each function call), it's better to allow users to implement the UDF function as static method which could potentially give better performance.

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

Yes. Spark users can now have the choice to define static magic method for `ScalarFunction` when it is written in Java and when the UDF is stateless.

### How was this patch tested?

Added new UT.

Closes #32407 from sunchao/SPARK-35261.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 20:34:51 -07:00
Chao Sun b4ec9e2304 [SPARK-35321][SQL] Don't register Hive permanent functions when creating Hive client
### What changes were proposed in this pull request?

Instantiate a new Hive client through `Hive.getWithFastCheck(conf, false)` instead of `Hive.get(conf)`.

### Why are the changes needed?

[HIVE-10319](https://issues.apache.org/jira/browse/HIVE-10319) introduced a new API `get_all_functions` which is only supported in Hive 1.3.0/2.0.0 and up. As result, when Spark 3.x talks to a HMS service of version 1.2 or lower, the following error will occur:
```
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.hadoop.hive.ql.metadata.Hive.getAllFunctions(Hive.java:3897)
        at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:248)
        at org.apache.hadoop.hive.ql.metadata.Hive.registerAllFunctionsOnce(Hive.java:231)
        ... 96 more
Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_all_functions'
        at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_all_functions(ThriftHiveMetastore.java:3845)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_all_functions(ThriftHiveMetastore.java:3833)
```

The `get_all_functions` is called only when `doRegisterAllFns` is set to true:
```java
  private Hive(HiveConf c, boolean doRegisterAllFns) throws HiveException {
    conf = c;
    if (doRegisterAllFns) {
      registerAllFunctionsOnce();
    }
  }
```

what this does is to register all Hive permanent functions defined in HMS in Hive's `FunctionRegistry` class, via iterating through results from `get_all_functions`. To Spark, this seems unnecessary as it loads Hive permanent (not built-in) UDF via directly calling the HMS API, i.e., `get_function`. The `FunctionRegistry` is only used in loading Hive's built-in function that is not supported by Spark. At this time, it only applies to `histogram_numeric`.

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

Yes with this fix Spark now should be able to talk to HMS server with Hive 1.2.x and lower (with HIVE-24608 too)

### How was this patch tested?

Manually started a HMS server of Hive version 1.2.2, with patched Hive 2.3.8 using HIVE-24608. Without the PR it failed with the above exception. With the PR the error disappeared and I can successfully perform common operations such as create table, create database, list tables, etc.

Closes #32446 from sunchao/SPARK-35321.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 15:06:04 -07:00
Liang-Chi Hsieh 33fbf5647b [SPARK-35288][SQL] StaticInvoke should find the method without exact argument classes match
### What changes were proposed in this pull request?

This patch proposes to make StaticInvoke able to find method with given method name even the parameter types do not exactly match to argument classes.

### Why are the changes needed?

Unlike `Invoke`, `StaticInvoke` only tries to get the method with exact argument classes. If the calling method's parameter types are not exactly matched with the argument classes, `StaticInvoke` cannot find the method.

`StaticInvoke` should be able to find the method under the cases too.

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

Yes. `StaticInvoke` can find a method even the argument classes are not exactly matched.

### How was this patch tested?

Unit test.

Closes #32413 from viirya/static-invoke.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-07 09:07:57 -07:00
beliefer d3b92eec45 [SPARK-35021][SQL] Group exception messages in connector/catalog
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32377 from beliefer/SPARK-35021.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 10:54:43 +00:00
Yingyi Bu 72d32662d4 [SPARK-35144][SQL] Migrate to transformWithPruning for object rules
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- APPEND_COLUMNS
- DESERIALIZE_TO_OBJECT
- LAMBDA_VARIABLE
- MAP_OBJECTS
- SERIALIZE_FROM_OBJECT
- PROJECT
- TYPED_FILTER

Added tree traversal pruning to the following rules dealing with objects:
- EliminateSerialization
- CombineTypedFilters
- EliminateMapObjects
- ObjectSerializerPruning

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32451 from sigmod/object.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-07 18:36:28 +08:00
Wenchen Fan 9aa18dfe19 [SPARK-35333][SQL] Skip object null check in Invoke if possible
### What changes were proposed in this pull request?

If `targetObject` is not nullable, we don't need the object null check in `Invoke`.

### Why are the changes needed?

small perf improvement

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

no

### How was this patch tested?

existing tests

Closes #32466 from cloud-fan/invoke.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 10:27:28 +00:00
gengjiaan cf2c4ba584 [SPARK-35020][SQL] Group exception messages in catalyst/util
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32367 from beliefer/SPARK-35020.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 08:30:30 +00:00
Wenchen Fan e83910f1f8 [SPARK-26164][SQL][FOLLOWUP] WriteTaskStatsTracker should know which file the row is written to
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/32198

Before https://github.com/apache/spark/pull/32198, in `WriteTaskStatsTracker.newRow`, we know that the row is written to the current file. After https://github.com/apache/spark/pull/32198 , we no longer know this connection.

This PR adds the file path parameter in `WriteTaskStatsTracker.newRow` to bring back the connection.

### Why are the changes needed?

To not break some custom `WriteTaskStatsTracker` implementations.

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

no

### How was this patch tested?

N/A

Closes #32459 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 08:28:42 +00:00
Terry Kim 33c1034315 [SPARK-34701][SQL][FOLLOW-UP] Children/innerChildren should be mutually exclusive for AnalysisOnlyCommand
### What changes were proposed in this pull request?

This is a follow up to https://github.com/apache/spark/pull/32032#discussion_r620928086. Basically, `children`/`innerChildren` should be mutually exclusive for `AlterViewAsCommand` and `CreateViewCommand`, which extend `AnalysisOnlyCommand`. Otherwise, there could be an issue in the `EXPLAIN` command. Currently, this is not an issue, because these commands will be analyzed (children will always be empty) when the `EXPLAIN` command is run.

### Why are the changes needed?

To be future-proof where these commands are directly used.

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

No.

### How was this patch tested?

Added new tsts

Closes #32447 from imback82/SPARK-34701-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-07 06:07:53 +00:00
Cheng Su 42f59caf73 [SPARK-35133][SQL] Explain codegen works with AQE
### What changes were proposed in this pull request?

`EXPLAIN CODEGEN <query>` (and Dataset.explain("codegen")) prints out the generated code for each stage of plan. The current implementation is to match `WholeStageCodegenExec` operator in query plan and prints out generated code (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala#L111-L118 ). This does not work with AQE as we wrap the whole query plan inside `AdaptiveSparkPlanExec` and do not run whole stage code-gen physical plan rule eagerly (`CollapseCodegenStages`). This introduces unexpected behavior change for EXPLAIN query (and Dataset.explain), as we enable AQE by default now.

The change is to explain code-gen for the current executed plan of AQE.

### Why are the changes needed?

Make `EXPLAIN CODEGEN` work same as before.

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

No (when comparing with latest Spark release 3.1.1).

### How was this patch tested?

Added unit test in `ExplainSuite.scala`.

Closes #32430 from c21/explain-aqe.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-06 20:44:31 -07:00
Yuanjian Li dfb3343423 [SPARK-34526][SS] Ignore the error when checking the path in FileStreamSink.hasMetadata
### What changes were proposed in this pull request?
When checking the path in `FileStreamSink.hasMetadata`, we should ignore the error and assume the user wants to read a batch output.

### Why are the changes needed?
Keep the original behavior of ignoring the error.

### Does this PR introduce _any_ user-facing change?
Yes.
The path checking will not throw an exception when checking file sink format

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

Closes #31638 from xuanyuanking/SPARK-34526.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-06 22:48:53 +09:00
Liang-Chi Hsieh 6cd5cf5722 [SPARK-35215][SQL] Update custom metric per certain rows and at the end of the task
### What changes were proposed in this pull request?

This patch changes custom metric updating to update per certain rows (currently 100), instead of per row.

### Why are the changes needed?

Based on previous discussion https://github.com/apache/spark/pull/31451#discussion_r605413557, we should only update custom metrics per certain (e.g. 100) rows and also at the end of the task. Updating per row doesn't make too much benefit.

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

No

### How was this patch tested?

Existing unit test.

Closes #32330 from viirya/metric-update.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-06 13:21:08 +00:00
Liang-Chi Hsieh c6d3f3778f [SPARK-35240][SS] Use CheckpointFileManager for checkpoint file manipulation
### What changes were proposed in this pull request?

This patch changes a few places using `FileSystem` API to manipulate checkpoint file to `CheckpointFileManager`.

### Why are the changes needed?

`CheckpointFileManager` is designed to handle checkpoint file manipulation. However, there are a few places exposing `FileSystem` from checkpoint files/paths. We should use `CheckpointFileManager` to manipulate checkpoint files. For example, we may want to have one storage system for checkpoint file. If all checkpoint file manipulation is performed through `CheckpointFileManager`, we can only implement `CheckpointFileManager` for the storage system, and don't need to implement `FileSystem` API for it.

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

No

### How was this patch tested?

Existing unit tests.

Closes #32361 from viirya/checkpoint-manager.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-06 00:49:37 -07:00
Linhong Liu 3f5a20919c [SPARK-35318][SQL] Hide internal view properties for describe table cmd
### What changes were proposed in this pull request?
Hide internal view properties for describe table command, because those
properties are generated by spark and should be transparent to the end-user.

### Why are the changes needed?
Avoid internal properties confusing the users.

### Does this PR introduce _any_ user-facing change?
Yes
Before this change, the user will see below output for `describe formatted test_view`
```
....
Table Properties       [view.catalogAndNamespace.numParts=2, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=default, view.query.out.col.0=c, view.query.out.col.1=v, view.query.out.numCols=2, view.referredTempFunctionsNames=[], view.referredTempViewNames=[]]
...
```
After this change, the internal properties will be hidden for `describe formatted test_view`
```
...
Table Properties        []
...
```

### How was this patch tested?
existing UT

Closes #32441 from linhongliu-db/hide-properties.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-06 07:31:34 +00:00
Takeshi Yamamuro 5c67d0c8f7 [SPARK-35293][SQL][TESTS] Use the newer dsdgen for TPCDSQueryTestSuite
### What changes were proposed in this pull request?

This PR intends to replace `maropu/spark-tpcds-datagen` with `databricks/tpcds-kit` for using a newer dsdgen and update the golden files in `tpcds-query-results`.

### Why are the changes needed?

For better testing.

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

No.

### How was this patch tested?

GA passed.

Closes #32420 from maropu/UseTpcdsKit.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-06 15:25:46 +09:00
Dongjoon Hyun 19661f6ae2 [SPARK-35325][SQL][TESTS] Add nested column ORC encryption test case
### What changes were proposed in this pull request?

This PR aims to enrich ORC encryption test coverage for nested columns.

### Why are the changes needed?

This will provide a test coverage for this feature.

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

No.

### How was this patch tested?

Pass the CIs with the newly added test case.

Closes #32449 from dongjoon-hyun/SPARK-35325.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-05 22:29:54 -07:00
Yingyi Bu 7970318296 [SPARK-35155][SQL] Add rule id pruning to Analyzer rules
### What changes were proposed in this pull request?

Added rule id based pruning to Analyzer rules in fixed point batches:

- org.apache.spark.sql.catalyst.analysis.Analyzer$AddMetadataColumns
- org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator
- org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractWindowExpressions
- org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggAliasInGroupBy
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveBinaryArithmetic
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveInsertInto
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOutputRelation
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolvePivot
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRandomSeed
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast
- org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUserSpecifiedColumns
- org.apache.spark.sql.catalyst.analysis.Analyzer$WindowsSubstitution
- org.apache.spark.sql.catalyst.analysis.DeduplicateRelations
- org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
- org.apache.spark.sql.catalyst.analysis.EliminateUnions
- org.apache.spark.sql.catalyst.analysis.ResolveCreateNamedStruct
- org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveCoalesceHints
- org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveJoinStrategyHints
- org.apache.spark.sql.catalyst.analysis.ResolveInlineTables
- org.apache.spark.sql.catalyst.analysis.ResolveLambdaVariables
- org.apache.spark.sql.catalyst.analysis.ResolveTimeZone
- org.apache.spark.sql.catalyst.analysis.ResolveUnion
- org.apache.spark.sql.catalyst.analysis.SubstituteUnresolvedOrdinals
- org.apache.spark.sql.catalyst.analysis.TimeWindowing

Subsequent PRs will add tree bits based pruning to those rules. Split a big PR to reduce review load.

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32425 from sigmod/analyzer.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-06 08:55:29 +08:00
Yijia Cui bbdbe0f734 [SPARK-34854][SQL][SS] Expose source metrics via progress report and add Kafka use-case to report delay
### What changes were proposed in this pull request?
This pull request proposes a new API for streaming sources to signal that they can report metrics, and adds a use case to support Kafka micro batch stream to report the stats of # of offsets for the current offset falling behind the latest.

A public interface is added.

`metrics`: returns the metrics reported by the streaming source with given offset.

### Why are the changes needed?
The new API can expose any custom metrics for the "current" offset for streaming sources. Different from #31398, this PR makes metrics available to user through progress report, not through spark UI. A use case is that people want to know how the current offset falls behind the latest offset.

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

### How was this patch tested?
Unit test for Kafka micro batch source v2 are added to test the Kafka use case.

Closes #31944 from yijiacui-db/SPARK-34297.

Authored-by: Yijia Cui <yijia.cui@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-05 17:26:07 +09:00
dsolow f550e03b96 [SPARK-34794][SQL] Fix lambda variable name issues in nested DataFrame functions
### What changes were proposed in this pull request?

To fix lambda variable name issues in nested DataFrame functions, this PR modifies code to use a global counter for `LambdaVariables` names created by higher order functions.

This is the rework of #31887. Closes #31887.

### Why are the changes needed?

 This moves away from the current hard-coded variable names which break on nested function calls. There is currently a bug where nested transforms in particular fail (the inner variable shadows the outer variable)

For this query:
```
val df = Seq(
    (Seq(1,2,3), Seq("a", "b", "c"))
).toDF("numbers", "letters")

df.select(
    f.flatten(
        f.transform(
            $"numbers",
            (number: Column) => { f.transform(
                $"letters",
                (letter: Column) => { f.struct(
                    number.as("number"),
                    letter.as("letter")
                ) }
            ) }
        )
    ).as("zipped")
).show(10, false)
```
This is the current (incorrect) output:
```
+------------------------------------------------------------------------+
|zipped                                                                  |
+------------------------------------------------------------------------+
|[{a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}]|
+------------------------------------------------------------------------+
```
And this is the correct output after fix:
```
+------------------------------------------------------------------------+
|zipped                                                                  |
+------------------------------------------------------------------------+
|[{1, a}, {1, b}, {1, c}, {2, a}, {2, b}, {2, c}, {3, a}, {3, b}, {3, c}]|
+------------------------------------------------------------------------+
```

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

No

### How was this patch tested?

Added the new test in `DataFrameFunctionsSuite`.

Closes #32424 from maropu/pr31887.

Lead-authored-by: dsolow <dsolow@sayari.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: dmsolow <dsolow@sayarianalytics.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-05 12:46:13 +09:00
Yingyi Bu 7fd3f8f9ec [SPARK-35294][SQL] Add tree traversal pruning in rules with dedicated files under optimizer
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- CREATE_NAMED_STRUCT
- EXTRACT_VALUE
- JSON_TO_STRUCT
- OUTER_REFERENCE
- AGGREGATE
- LOCAL_RELATION
- EXCEPT
- LIMIT
- WINDOW

Used them in the following rules:
- DecorrelateInnerQuery
- LimitPushDownThroughWindow
- OptimizeCsvJsonExprs
- PropagateEmptyRelation
- PullOutGroupingExpressions
- PushLeftSemiLeftAntiThroughJoin
- ReplaceExceptWithFilter
- RewriteDistinctAggregates
- SimplifyConditionalsInPredicate
- UnwrapCastInBinaryComparison

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32421 from sigmod/opt.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-04 19:17:22 +08:00
HyukjinKwon 8aaa9e890a [SPARK-35250][SQL][DOCS] Fix duplicated STOP_AT_DELIMITER to SKIP_VALUE at CSV's unescapedQuoteHandling option documentation
### What changes were proposed in this pull request?

This is rather a followup of https://github.com/apache/spark/pull/30518 that should be ported back to `branch-3.1` too.
`STOP_AT_DELIMITER` was mistakenly used twice. The duplicated `STOP_AT_DELIMITER` should be `SKIP_VALUE` in the documentation.

### Why are the changes needed?

To correctly document.

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

Yes, it fixes the user-facing documentation.

### How was this patch tested?

I checked them via running linters.

Closes #32423 from HyukjinKwon/SPARK-35250.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-04 08:44:18 +09:00
Tobias Hermann 54e0aa10c8 [MINOR][SS][DOCS] Fix a typo in the documentation of GroupState
### What changes were proposed in this pull request?

Fixing some typos in the documenting comments.

### Why are the changes needed?

To make reading the docs more pleasant.

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

Yes, since the user sees the docs.

### How was this patch tested?

It was not tested, because no code was changed.

Closes #32400 from Dobiasd/patch-1.

Authored-by: Tobias Hermann <editgym@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 19:35:38 +09:00
Chao Sun 2a8d7ed4bf [SPARK-35281][SQL] StaticInvoke should not apply boxing if return type is primitive
### What changes were proposed in this pull request?

In `StaticInvoke`, when result is nullable, don't box the return value if its type is primitive.

### Why are the changes needed?

It is unnecessary to apply boxing when the method return value is of primitive type, and it would hurt performance a lot if the method is simple. The check is done in `Invoke` but not in `StaticInvoke`.

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

No

### How was this patch tested?

Added a UT.

Closes #32416 from sunchao/SPARK-35281.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 14:55:35 +09:00
Max Gekk 335f00b19b [SPARK-35285][SQL] Parse ANSI interval types in SQL schema
### What changes were proposed in this pull request?
1. Extend Spark SQL parser to support parsing of:
    - `INTERVAL YEAR TO MONTH` to `YearMonthIntervalType`
    - `INTERVAL DAY TO SECOND` to `DayTimeIntervalType`
2. Assign new names to the ANSI interval types according to the SQL standard to be able to parse the names back by Spark SQL parser. Override the `typeName()` name of `YearMonthIntervalType`/`DayTimeIntervalType`.

### Why are the changes needed?
To be able to use new ANSI interval types in SQL. The SQL standard requires the types to be defined according to the rules:
```
<interval type> ::= INTERVAL <interval qualifier>
<interval qualifier> ::= <start field> TO <end field> | <single datetime field>
<start field> ::= <non-second primary datetime field> [ <left paren> <interval leading field precision> <right paren> ]
<end field> ::= <non-second primary datetime field> | SECOND [ <left paren> <interval fractional seconds precision> <right paren> ]
<primary datetime field> ::= <non-second primary datetime field | SECOND
<non-second primary datetime field> ::= YEAR | MONTH | DAY | HOUR | MINUTE
<interval fractional seconds precision> ::= <unsigned integer>
<interval leading field precision> ::= <unsigned integer>
```
Currently, Spark SQL supports only `YEAR TO MONTH` and `DAY TO SECOND` as `<interval qualifier>`.

### Does this PR introduce _any_ user-facing change?
Should not since the types has not been released yet.

### How was this patch tested?
By running the affected tests such as:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "test:testOnly *ExpressionTypeCheckingSuite"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z windowFrameCoercion.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z literals.sql"
```

Closes #32409 from MaxGekk/parse-ansi-interval-types.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 13:50:35 +09:00
Takeshi Yamamuro cd689c942c [SPARK-35192][SQL][TESTS] Port minimal TPC-DS datagen code from databricks/spark-sql-perf
### What changes were proposed in this pull request?

This PR proposes to port minimal code to generate TPC-DS data from [databricks/spark-sql-perf](https://github.com/databricks/spark-sql-perf). The classes in a new class file `tpcdsDatagen.scala` are basically copied from the `databricks/spark-sql-perf` codebase.
Note that I've modified them a bit to follow the Spark code style and removed unnecessary parts from them.

The code authors of these classes are:
juliuszsompolski
npoggi
wangyum

### Why are the changes needed?

We frequently use TPCDS data now for benchmarks/tests, but the classes for the TPCDS schemas of datagen and benchmarks/tests are managed separately, e.g.,
 - https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala
 - https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/tpcds/TPCDSTables.scala

 I think this causes some inconveniences, e.g., we need to update both files in the separate repositories if we update the TPCDS schema #32037. So, it would be useful for the Spark codebase to generate them by referring to the same schema definition.

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

dev only.

### How was this patch tested?

Manually checked and GA passed.

Closes #32243 from maropu/tpcdsDatagen.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-03 12:04:42 +09:00
Angerszhuuuu caa46ce0b6 [SPARK-35112][SQL] Support Cast string to day-second interval
### What changes were proposed in this pull request?
Support Cast string to day-seconds interval

### Why are the changes needed?
Users can cast day-second interval string to DayTimeIntervalType.

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

### How was this patch tested?
Added UT

Closes #32271 from AngersZhuuuu/SPARK-35112.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-02 09:28:51 +03:00
Peter Toth cfc0495f9c [SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function
### What changes were proposed in this pull request?
This PR adds a new rule `PullOutGroupingExpressions` to pull out complex grouping expressions to a `Project` node under an `Aggregate`. These expressions are then referenced in both grouping expressions and aggregate expressions without aggregate functions to ensure that optimization rules don't change the aggregate expressions to invalid ones that no longer refer to any grouping expressions.

### Why are the changes needed?
If aggregate expressions (without aggregate functions) in an `Aggregate` node are complex then the `Optimizer` can optimize out grouping expressions from them and so making aggregate expressions invalid.

Here is a simple example:
```
SELECT not(t.id IS NULL) , count(*)
FROM t
GROUP BY t.id IS NULL
```
In this case the `BooleanSimplification` rule does this:
```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.BooleanSimplification ===
!Aggregate [isnull(id#222)], [NOT isnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]   Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]
 +- Project [value#219 AS id#222]                                                                 +- Project [value#219 AS id#222]
    +- LocalRelation [value#219]                                                                     +- LocalRelation [value#219]
```
where `NOT isnull(id#222)` is optimized to `isnotnull(id#222)` and so it no longer refers to any grouping expression.

Before this PR:
```
== Optimized Logical Plan ==
Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#234, count(1) AS c#232L]
+- Project [value#219 AS id#222]
   +- LocalRelation [value#219]
```
and running the query throws an error:
```
Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
java.lang.IllegalStateException: Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
```

After this PR:
```
== Optimized Logical Plan ==
Aggregate [_groupingexpression#233], [NOT _groupingexpression#233 AS (NOT (id IS NULL))#230, count(1) AS c#228L]
+- Project [isnull(value#219) AS _groupingexpression#233]
   +- LocalRelation [value#219]
```
and the query works.

### Does this PR introduce _any_ user-facing change?
Yes, the query works.

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

Closes #32396 from peter-toth/SPARK-34581-keep-grouping-expressions-2.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-02 05:53:09 +00:00
Liang-Chi Hsieh 6ce1b161e9 [SPARK-35278][SQL] Invoke should find the method with correct number of parameters
### What changes were proposed in this pull request?

This patch fixes `Invoke` expression when the target object has more than one method with the given method name.

### Why are the changes needed?

`Invoke` will find out the method on the target object with given method name. If there are more than one method with the name, currently it is undeterministic which method will be used. We should add the condition of parameter number when finding the method.

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

Yes, fixed a bug when using `Invoke` on a object where more than one method with the given method name.

### How was this patch tested?

Unit test.

Closes #32404 from viirya/verify-invoke-param-len.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-05-01 10:20:46 -07:00
Yuming Wang 72e238a790 [SPARK-35273][SQL] CombineFilters support non-deterministic expressions
### What changes were proposed in this pull request?

This pr makes `CombineFilters` support non-deterministic expressions. For example:
```sql
spark.sql("CREATE TABLE t1(id INT, dt STRING) using parquet PARTITIONED BY (dt)")
spark.sql("CREATE VIEW v1 AS SELECT * FROM t1 WHERE dt NOT IN ('2020-01-01', '2021-01-01')")
spark.sql("SELECT * FROM v1 WHERE dt = '2021-05-01' AND rand() <= 0.01").explain()
```

Before this pr:
```
== Physical Plan ==
*(1) Filter (isnotnull(dt#1) AND ((dt#1 = 2021-05-01) AND (rand(-6723800298719475098) <= 0.01)))
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[id#0,dt#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(0 paths)[], PartitionFilters: [NOT dt#1 IN (2020-01-01,2021-01-01)], PushedFilters: [], ReadSchema: struct<id:int>
```

After this pr:
```
== Physical Plan ==
*(1) Filter (rand(-2400509328955813273) <= 0.01)
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[id#0,dt#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(0 paths)[], PartitionFilters: [isnotnull(dt#1), NOT dt#1 IN (2020-01-01,2021-01-01), (dt#1 = 2021-05-01)], PushedFilters: [], ReadSchema: struct<id: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 #32405 from wangyum/SPARK-35273.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-01 06:02:11 +00:00
ulysses-you 39889df32a [SPARK-35264][SQL] Support AQE side broadcastJoin threshold
### What changes were proposed in this pull request?

~~This PR aims to add a new AQE optimizer rule `DynamicJoinSelection`. Like other AQE partition number configs, this rule add a new broadcast threshold config `spark.sql.adaptive.autoBroadcastJoinThreshold`.~~
This PR amis to add a flag in `Statistics` to distinguish AQE stats or normal stats, so that we can make some sql configs isolation between AQE and normal.

### Why are the changes needed?

The main idea here is that make join config isolation between normal planner and aqe planner which shared the same code path.

Actually we do not very trust using the static stats to consider if it can build broadcast hash join. In our experience it's very common that Spark throw broadcast timeout or driver side OOM exception when execute a bit large plan. And due to braodcast join is not reversed which means if we covert join to braodcast hash join at first time, we(AQE) can not optimize it again, so it should make sense to decide if we can do broadcast at aqe side using different sql config.

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

Yes, a new config `spark.sql.adaptive.autoBroadcastJoinThreshold` added.

### How was this patch tested?

Add new test.

Closes #32391 from ulysses-you/SPARK-35264.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-30 09:16:21 +00:00
Angerszhuuuu 11ea255283 [SPARK-35111][SQL] Support Cast string to year-month interval
### What changes were proposed in this pull request?
Support Cast string to year-month interval
Supported format as below
```
ANSI_STYLE, like
INTERVAL -'-10-1' YEAR TO MONTH
HIVE_STYLE like
10-1 or -10-1

Rules from the SQL standard about ANSI_STYLE:

<interval literal> ::=
  INTERVAL [ <sign> ] <interval string> <interval qualifier>
<interval string> ::=
  <quote> <unquoted interval string> <quote>
<unquoted interval string> ::=
  [ <sign> ] { <year-month literal> | <day-time literal> }
<year-month literal> ::=
  <years value> [ <minus sign> <months value> ]
  | <months value>
<years value> ::=
  <datetime value>
<months value> ::=
  <datetime value>
<datetime value> ::=
  <unsigned integer>
<unsigned integer> ::= <digit>...
```
### Why are the changes needed?
Support Cast string to year-month interval

### Does this PR introduce _any_ user-facing change?
User can cast year month interval string to YearMonthIntervalType

### How was this patch tested?
Added UT

Closes #32266 from AngersZhuuuu/SPARK-SPARK-35111.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-30 08:03:07 +03:00
Kousuke Saruta e8bf8fe213 [SPARK-35047][SQL] Allow Json datasources to write non-ascii characters as codepoints
### What changes were proposed in this pull request?

This PR proposes to enable the JSON datasources to write non-ascii characters as codepoints.
To enable/disable this feature, I introduce a new option `writeNonAsciiCharacterAsCodePoint` for JSON datasources.

### Why are the changes needed?

JSON specification allows codepoints as literal but Spark SQL's JSON datasources don't support the way to do it.
It's great if we can write non-ascii characters as codepoints, which is a platform neutral representation.

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

Yes. Users can write non-ascii characters as codepoints with JSON datasources.

### How was this patch tested?

New test.

Closes #32147 from sarutak/json-unicode-write.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-29 09:50:15 -07:00
Kousuke Saruta 132cbf0c8c [SPARK-35105][SQL] Support multiple paths for ADD FILE/JAR/ARCHIVE commands
### What changes were proposed in this pull request?

This PR extends `ADD FILE/JAR/ARCHIVE` commands to be able to take multiple path arguments like Hive.

### Why are the changes needed?

To make those commands more useful.

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

Yes. In the current implementation, those commands can take a path which contains whitespaces without enclose it by neither `'` nor `"` but after this change, users need to enclose such paths.
I've note this incompatibility in the migration guide.

### How was this patch tested?

New tests.

Closes #32205 from sarutak/add-multiple-files.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-29 13:58:51 +09:00
Kousuke Saruta 529b875901 [SPARK-35226][SQL] Support refreshKrb5Config option in JDBC datasources
### What changes were proposed in this pull request?

This PR proposes to introduce a new JDBC option `refreshKrb5Config` which allows to reflect the change of `krb5.conf`.

### Why are the changes needed?

In the current master, JDBC datasources can't accept `refreshKrb5Config` which is defined in `Krb5LoginModule`.
So even if we change the `krb5.conf` after establishing a connection, the change will not be reflected.

The similar issue happens when we run multiple `*KrbIntegrationSuites` at the same time.
`MiniKDC` starts and stops every KerberosIntegrationSuite and different port number is recorded to `krb5.conf`.
Due to `SecureConnectionProvider.JDBCConfiguration` doesn't take `refreshKrb5Config`, KerberosIntegrationSuites except the first running one see the wrong port so those suites fail.
You can easily confirm with the following command.
```
build/sbt -Phive Phive-thriftserver -Pdocker-integration-tests "testOnly org.apache.spark.sql.jdbc.*KrbIntegrationSuite"
```
### Does this PR introduce _any_ user-facing change?

Yes. Users can set `refreshKrb5Config` to refresh krb5 relevant configuration.

### How was this patch tested?

New test.

Closes #32344 from sarutak/kerberos-refresh-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-29 13:55:53 +09:00
Kent Yao 771356555c [SPARK-34786][SQL][FOLLOWUP] Explicitly declare DecimalType(20, 0) for Parquet UINT_64
### What changes were proposed in this pull request?

Explicitly declare DecimalType(20, 0) for Parquet UINT_64, avoid use DecimalType.LongDecimal which only happens to have 20 as precision.

https://github.com/apache/spark/pull/31960#discussion_r622691560

### Why are the changes needed?

fix ambiguity

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

no

### How was this patch tested?

not needed, just current CI pass

Closes #32390 from yaooqinn/SPARK-34786-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-29 04:51:27 +00:00
Wenchen Fan 403e4795e9 [SPARK-35244][SQL][FOLLOWUP] Add null check for the exception cause
### What changes were proposed in this pull request?

Make sure we re-throw an exception that is not null.

### Why are the changes needed?

to be super safe

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

no

### How was this patch tested?

N/A

Closes #32387 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-29 09:21:32 +09:00
Chao Sun 86d3bb5f7d [SPARK-34981][SQL] Implement V2 function resolution and evaluation
Co-Authored-By: Chao Sun <sunchaoapple.com>
Co-Authored-By: Ryan Blue <rbluenetflix.com>

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

This implements function resolution and evaluation for functions registered through V2 FunctionCatalog [SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658). In particular:
- Added documentation for how to define the "magic method" in `ScalarFunction`.
- Added a new expression `ApplyFunctionExpression` which evaluates input by delegating to `ScalarFunction.produceResult` method.
- added a new expression `V2Aggregator` which is a type of `TypedImperativeAggregate`. It's a wrapper of V2 `AggregateFunction` and mostly delegate methods to the implementation of the latter. It also uses plain Java serde for intermediate state.
- Added function resolution logic for `ScalarFunction` and `AggregateFunction` in `Analyzer`.
  + For `ScalarFunction` this checks if the magic method is implemented through Java reflection, and create a `Invoke` expression if so. Otherwise, it checks if the default `produceResult` is overridden. If so, it creates a `ApplyFunctionExpression` which evaluates through `InternalRow`. Otherwise an analysis exception is thrown.
 + For `AggregateFunction`, this checks if the `update` method is overridden. If so, it converts it to `V2Aggregator`. Otherwise an analysis exception is thrown similar to the case of `ScalarFunction`.
- Extended existing `InMemoryTableCatalog` to add the function catalog capability. Also renamed it to `InMemoryCatalog` since it no longer only covers tables.

**Note**: this currently can successfully detect whether a subclass overrides the default `produceResult` or `update` method from the parent interface **only for Java implementations**. It seems in Scala it's hard to differentiate whether a subclass overrides a default method from its parent interface. In this case, it will be a runtime error instead of analysis error.

A few TODOs:
- Extend `V2SessionCatalog` with function catalog. This seems a little tricky since API such V2 `FunctionCatalog`'s `loadFunction` is different from V1 `SessionCatalog`'s `lookupFunction`.
- Add magic method for `AggregateFunction`.
- Type coercion when looking up functions

### Why are the changes needed?

As V2 FunctionCatalog APIs are finalized, we should integrate it with function resolution and evaluation process so that they are actually useful.

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

Yes, now a function exposed through V2 FunctionCatalog can be analyzed and evaluated.

### How was this patch tested?

Added new unit tests.

Closes #32082 from sunchao/resolve-func-v2.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 17:21:49 +00:00
ulysses-you 0bcf348438 [SPARK-34781][SQL][FOLLOWUP] Adjust the order of AQE optimizer rules
### What changes were proposed in this pull request?

Reorder  `DemoteBroadcastHashJoin` and `EliminateUnnecessaryJoin`.

### Why are the changes needed?

Skip unnecessary check in `DemoteBroadcastHashJoin` if `EliminateUnnecessaryJoin` affects.

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

No

### How was this patch tested?

No result affect.

Closes #32380 from ulysses-you/SPARK-34781-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 13:59:24 +00:00
ulysses-you 8b62c2964d [SPARK-35214][SQL] OptimizeSkewedJoin support ShuffledHashJoinExec
### What changes were proposed in this pull request?

Add `ShuffledHashJoin` pattern check in `OptimizeSkewedJoin` so that we can optimize it.

### Why are the changes needed?

Currently, we have already supported all type of join through hint that make it easy to choose the join implementation.

We would choose `ShuffledHashJoin` if one table is not big but over the broadcast threshold. It's better that we can support optimize it in `OptimizeSkewedJoin`.

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

Probably yes, the execute plan in AQE mode may be changed.

### How was this patch tested?

Improve exists test in `AdaptiveQueryExecSuite`

Closes #32328 from ulysses-you/SPARK-35214.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-28 16:57:57 +09:00
gengjiaan 56bb8155c5 [SPARK-35085][SQL] Get columns operation should handle ANSI interval column properly
### What changes were proposed in this pull request?
This PR let JDBC clients identify ANSI interval columns properly.

### Why are the changes needed?
This PR is similar to https://github.com/apache/spark/pull/29539.
JDBC users can query interval values through thrift server, create views with ansi interval columns, e.g.
`CREATE global temp view view1 as select interval '1-1' year to month as I;`
but when they want to get the details of the columns of view1, the will fail with `Unrecognized type name: YEAR-MONTH INTERVAL`
```
Caused by: java.lang.IllegalArgumentException: Unrecognized type name: YEAR-MONTH INTERVAL
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.toJavaSQLType(SparkGetColumnsOperation.scala:190)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$addToRowSet$1(SparkGetColumnsOperation.scala:206)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.addToRowSet(SparkGetColumnsOperation.scala:198)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$7(SparkGetColumnsOperation.scala:109)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$7$adapted(SparkGetColumnsOperation.scala:109)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$5(SparkGetColumnsOperation.scala:109)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$5$adapted(SparkGetColumnsOperation.scala:107)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.runInternal(SparkGetColumnsOperation.scala:107)
	... 34 more
```

### Does this PR introduce _any_ user-facing change?
Yes. Let hive JDBC recognize ANSI interval.

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

Closes #32345 from beliefer/SPARK-35085.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-28 08:58:43 +03:00
PengLei 046c8c3dd6 [SPARK-34878][SQL][TESTS] Check actual sizes of year-month and day-time intervals
### What changes were proposed in this pull request?
As we have suport the year-month and day-time intervals.  Add the test actual size of year-month and day-time intervals type

### Why are the changes needed?
Just add test

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

### How was this patch tested?
./dev/scalastyle
run test for "ColumnTypeSuite"

Closes #32366 from Peng-Lei/SPARK-34878.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-28 07:48:49 +03:00
Jose Torres 253a1aee46 [SPARK-35246][SS] Don't allow streaming-batch intersects
### What changes were proposed in this pull request?
The UnsupportedOperationChecker shouldn't allow streaming-batch intersects. As described in the ticket, they can't actually be planned correctly, and even simple cases like the below will fail:

```
  test("intersect") {
    val input = MemoryStream[Long]
    val df = input.toDS().intersect(spark.range(10).as[Long])
    testStream(df) (
      AddData(input, 1L),
      CheckAnswer(1)
    )
  }
```

### Why are the changes needed?
Users will be confused by the cryptic errors produced from trying to run an invalid query plan.

### Does this PR introduce _any_ user-facing change?
Some queries which previously failed with a poor error will now fail with a better one.

### How was this patch tested?
modified unit test

Closes #32371 from jose-torres/ossthing.

Authored-by: Jose Torres <joseph.torres@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-28 10:47:11 +09:00
Wenchen Fan 10c2b68d24 [SPARK-35244][SQL] Invoke should throw the original exception
### What changes were proposed in this pull request?

This PR updates the interpreted code path of invoke expressions, to unwrap the `InvocationTargetException`

### Why are the changes needed?

Make interpreted and codegen path consistent for invoke expressions.

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

no

### How was this patch tested?

new UT

Closes #32370 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-28 10:45:04 +09:00
Kousuke Saruta abb1f0c5d7 [SPARK-35236][SQL] Support archive files as resources for CREATE FUNCTION USING syntax
### What changes were proposed in this pull request?

This PR proposes to make `CREATE FUNCTION USING` syntax can take archives as resources.

### Why are the changes needed?

It would be useful.
`CREATE FUNCTION USING` syntax doesn't support archives as resources because archives were not supported in Spark SQL.
Now Spark SQL supports archives so I think we can support them for the syntax.

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

Yes. Users can specify archives for `CREATE FUNCTION USING` syntax.

### How was this patch tested?

New test.

Closes #32359 from sarutak/load-function-using-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-28 10:15:21 +09:00
Kent Yao 16d223efee [SPARK-35091][SPARK-35090][SQL] Support extract from ANSI Intervals
### What changes were proposed in this pull request?

In this PR, we add extract/date_part support for ANSI Intervals

The `extract` is an ANSI expression and `date_part` is NON-ANSI but exists as an equivalence for `extract`

#### expression

```
<extract expression> ::=
  EXTRACT <left paren> <extract field> FROM <extract source> <right paren>
```

#### <extract field> for interval source

```

<primary datetime field> ::=
    <non-second primary datetime field>
| SECOND
<non-second primary datetime field> ::=
    YEAR
  | MONTH
  | DAY
  | HOUR
  | MINUTE
```

#### dataType

```
If <extract field> is a <primary datetime field> that does not specify SECOND or <extract field> is not a <primary datetime field>, then the declared type of the result is an implementation-defined exact numeric type with scale 0 (zero)

Otherwise, the declared type of the result is an implementation-defined exact numeric type with scale not less than the specified or implied <time fractional seconds precision> or <interval fractional seconds precision>, as appropriate, of the SECOND <primary datetime field> of the <extract source>.
```

### Why are the changes needed?

Subtask of ANSI Intervals Support

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

Yes
1. extract/date_part support ANSI intervals
2. for non-ansi intervals, the return type is changed from long to byte when extracting hours

### How was this patch tested?

new added tests

Closes #32351 from yaooqinn/SPARK-35091.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:06:54 +00:00
ulysses-you 4ff9f1fe3b [SPARK-35239][SQL] Coalesce shuffle partition should handle empty input RDD
### What changes were proposed in this pull request?

Create empty partition for custom shuffle reader if input RDD is empty.

### Why are the changes needed?

If input RDD partition is empty then the map output statistics will be null. And if all shuffle stage's input RDD partition is empty, we will skip it and lose the chance to coalesce partition.

We can simply create a empty partition for these custom shuffle reader to reduce the partition number.

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

Yes, the shuffle partition might be changed in AQE.

### How was this patch tested?

add new test.

Closes #32362 from ulysses-you/SPARK-35239.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:05:57 +00:00
gengjiaan 55dea2d937 [SPARK-34837][SQL][FOLLOWUP] Fix division by zero in the avg function over ANSI intervals
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/32229 support ANSI SQL intervals by the aggregate function `avg`.
But have not treat that the input zero rows. so this will lead to:
```
Caused by: java.lang.ArithmeticException: / by zero
	at com.google.common.math.LongMath.divide(LongMath.java:367)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1864)
	at org.apache.spark.rdd.RDD.$anonfun$count$1(RDD.scala:1253)
	at org.apache.spark.rdd.RDD.$anonfun$count$1$adapted(RDD.scala:1253)
	at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2248)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
```

### Why are the changes needed?
Fix a bug.

### Does this PR introduce _any_ user-facing change?
No. Just new feature.

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

Closes #32358 from beliefer/SPARK-34837-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-27 10:52:12 +03:00
Angerszhuuuu 2d2f467831 [SPARK-35169][SQL] Fix wrong result of min ANSI interval division by -1
### What changes were proposed in this pull request?
Before this patch
```
scala> Seq(java.time.Period.ofMonths(Int.MinValue)).toDF("i").select($"i" / -1).show(false)
+-------------------------------------+
|(i / -1)                             |
+-------------------------------------+
|INTERVAL '-178956970-8' YEAR TO MONTH|
+-------------------------------------+
scala> Seq(java.time.Duration.of(Long.MinValue, java.time.temporal.ChronoUnit.MICROS)).toDF("i").select($"i" / -1).show(false)
+---------------------------------------------------+
|(i / -1)                                           |
+---------------------------------------------------+
|INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND|
+---------------------------------------------------+
```

Wrong result of min ANSI interval division by -1, this pr fix this

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32314 from AngersZhuuuu/SPARK-35169.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 07:05:50 +00:00
Cheng Su c4ad86f311 [SPARK-35235][SQL][TEST] Add row-based hash map into aggregate benchmark
### What changes were proposed in this pull request?

`AggregateBenchmark` is only testing the performance for vectorized fast hash map, but not row-based hash map (which is used by default). We should add the row-based hash map into the benchmark.

java 8 benchmark run - https://github.com/c21/spark/actions/runs/787731549
java 11 benchmark run - https://github.com/c21/spark/actions/runs/787742858

### Why are the changes needed?

To have and track a basic sense of benchmarking different fast hash map used in hash aggregate.

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

No.

### How was this patch tested?

Existing unit test, as this only touches benchmark code.

Closes #32357 from c21/agg-benchmark.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 06:53:42 +00:00
PengLei eb08b9010a [SPARK-35139][SQL] Support ANSI intervals as Arrow Column vectors
### What changes were proposed in this pull request?
 Support YearMonthIntervalType and DayTimeIntervalType to extend ArrowColumnVector

### Why are the changes needed?
https://issues.apache.org/jira/browse/SPARK-35139

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

### How was this patch tested?
1. By checking coding style via:
    $ ./dev/scalastyle
    $ ./dev/lint-java
2. Run the test "ArrowWriterSuite"

Closes #32340 from Peng-Lei/SPARK-35139.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 06:08:17 +00:00
Cheng Su 7f51106c0d [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table
### What changes were proposed in this pull request?

This is a re-proposal of https://github.com/apache/spark/pull/23163. Currently spark always requires a [local sort](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala#L188) before writing to output table with dynamic partition/bucket columns. The sort can be unnecessary if cardinality of partition/bucket values is small, and can be avoided by keeping multiple output writers concurrently.

This PR introduces a config `spark.sql.maxConcurrentOutputFileWriters` (which disables this feature by default), where user can tune the maximal number of concurrent writers. The config is needed here as we cannot keep arbitrary number of writers in task memory which can cause OOM (especially for Parquet/ORC vectorization writer).

The feature is to first use concurrent writers to write rows. If the number of writers exceeds the above config specified limit. Sort rest of rows and write rows one by one (See `DynamicPartitionDataConcurrentWriter.writeWithIterator()`).

In addition, interface `WriteTaskStatsTracker` and its implementation `BasicWriteTaskStatsTracker` are also changed because previously they are relying on the assumption that only one writer is active for writing dynamic partitions and bucketed table.

### Why are the changes needed?

Avoid the sort before writing output for dynamic partitioned query and bucketed table.
Help improve CPU and IO performance for these queries.

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

No.

### How was this patch tested?

Added unit test in `DataFrameReaderWriterSuite.scala`.

Closes #32198 from c21/writer.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 05:37:08 +00:00
Terry Kim 7779fce79a [SPARK-35225][SQL] EXPLAIN command should handle empty output of analyzed plan
### What changes were proposed in this pull request?

EXPLAIN command puts an empty line if there is no output for an analyzed plan. For example,

`sql("CREATE VIEW test AS SELECT 1").explain(true)` produces:
```
== Parsed Logical Plan ==
'CreateViewStatement [test], SELECT 1, false, false, PersistedView
+- 'Project [unresolvedalias(1, None)]
   +- OneRowRelation

== Analyzed Logical Plan ==

CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation

== Optimized Logical Plan ==
CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation

== Physical Plan ==
Execute CreateViewCommand
   +- CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
         +- Project [1 AS 1#7]
            +- OneRowRelation
```

### Why are the changes needed?

To handle empty output of analyzed plan and remove the unneeded empty line.

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

Yes, now the EXPLAIN command for the analyzed plan produces the following without the empty line:
```
== Analyzed Logical Plan ==
CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation
```

### How was this patch tested?

Added a test.

Closes #32342 from imback82/analyzed_plan_blank_line.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-27 11:10:16 +09:00
Shixiong Zhu 0df3b501ae [SPARK-28247][SS][TEST] Fix flaky test "query without test harness" on ContinuousSuite
### What changes were proposed in this pull request?

This is another attempt to fix the flaky test "query without test harness" on ContinuousSuite.

`query without test harness` is flaky because it starts a continuous query with two partitions but assumes they will run at the same speed.

In this test, 0 and 2 will be written to partition 0, 1 and 3 will be written to partition 1. It assumes when we see 3, 2 should be written to the memory sink. But this is not guaranteed. We can add `if (currentValue == 2) Thread.sleep(5000)` at this line b2a2b5d820/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala (L135) to reproduce the failure: `Result set Set([0], [1], [3]) are not a superset of Set(0, 1, 2, 3)!`

The fix is changing `waitForRateSourceCommittedValue` to wait until all partitions reach the desired values before stopping the query.

### Why are the changes needed?

Fix a flaky test.

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

No

### How was this patch tested?

Existing tests. Manually verify the reproduction I mentioned above doesn't fail after this change.

Closes #32316 from zsxwing/SPARK-28247-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-04-27 08:07:09 +09:00
beliefer 1b609c7dcf [SPARK-35060][SQL] Group exception messages in sql/types
### What changes were proposed in this pull request?
This PR group exception messages in `sql/catalyst/src/main/scala/org/apache/spark/sql/types`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #32244 from beliefer/SPARK-35060.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-26 16:44:51 +00:00
Angerszhuuuu f0090463a8 [SPARK-33985][SQL][TESTS] Add query test of combine usage of TRANSFORM and CLUSTER BY/ORDER BY
### What changes were proposed in this pull request?
Under hive's document  https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Transform there are many usage about  TRANSFORM and CLUSTER BY/ORDER BY, in this pr add some test about this cases.

### Why are the changes needed?
Add UT

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

### How was this patch tested?
Added UT

Closes #32333 from AngersZhuuuu/SPARK-33985.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-26 16:42:07 +00:00
Liang-Chi Hsieh c59988aa79 [SPARK-34638][SQL] Single field nested column prune on generator output
### What changes were proposed in this pull request?

This patch proposes an improvement on nested column pruning if the pruning target is generator's output. Previously we disallow such case. This patch allows to prune on it if there is only one single nested column is accessed after `Generate`.

E.g., `df.select(explode($"items").as('item)).select($"item.itemId")`. As we only need `itemId` from `item`, we can prune other fields out and only keep `itemId`.

In this patch, we only address explode-like generators. We will address other generators in followups.

### Why are the changes needed?

This helps to extend the availability of nested column pruning.

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

No

### How was this patch tested?

Unit test

Closes #31966 from viirya/SPARK-34638.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-26 09:32:22 -07:00
Liang-Chi Hsieh bdac19184a [SPARK-35230][SQL] Move custom metric classes to proper package
### What changes were proposed in this pull request?

This patch moves DS v2 custom metric classes to `org.apache.spark.sql.connector.metric` package. Moving `CustomAvgMetric` and `CustomSumMetric` to above package and make them as public java abstract class too.

### Why are the changes needed?

`CustomAvgMetric` and `CustomSumMetric`  should be public APIs for developers to extend. As there are a few metric classes, we should put them together in one package.

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

No, dev only and they are not released yet.

### How was this patch tested?

Unit tests.

Closes #32348 from viirya/move-custom-metric-classes.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-26 07:19:36 -07:00
beliefer c0a3c0cbbe [SPARK-35088][SQL] Accept ANSI intervals by the Sequence expression
### What changes were proposed in this pull request?
This PR makes `Sequence` expression supports ANSI intervals as step expression.
If the start and stop expression is `TimestampType,` then the step expression could select year-month or day-time interval.
If the start and stop expression is `DateType,` then the step expression must be year-month.

### Why are the changes needed?
Extends the function of `Sequence` expression.

### Does this PR introduce _any_ user-facing change?
'Yes'. Users could use ANSI intervals as step expression for `Sequence` expression.

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

Closes #32311 from beliefer/SPARK-35088.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-26 10:05:57 +03:00
Adam Binford 74afc68e21 [SPARK-35213][SQL] Keep the correct ordering of nested structs in chained withField operations
### What changes were proposed in this pull request?

Modifies the UpdateFields optimizer to fix correctness issues with certain nested and chained withField operations. Examples for recreating the issue are in the new unit tests as well as the JIRA issue.

### Why are the changes needed?

Certain withField patterns can cause Exceptions or even incorrect results. It appears to be a result of the additional UpdateFields optimization added in https://github.com/apache/spark/pull/29812. It traverses fieldOps in reverse order to take the last one per field, but this can cause nested structs to change order which leads to mismatches between the schema and the actual data. This updates the optimization to maintain the initial ordering of nested structs to match the generated schema.

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

It fixes exceptions and incorrect results for valid uses in the latest Spark release.

### How was this patch tested?

Added new unit tests for these edge cases.

Closes #32338 from Kimahriman/bug/optimize-with-fields.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-25 23:39:56 -07:00
Max Gekk d572a85989 [SPARK-35224][SQL][TESTS] Fix buffer overflow in MutableProjectionSuite
### What changes were proposed in this pull request?
In the test `"unsafe buffer with NO_CODEGEN"` of `MutableProjectionSuite`, fix unsafe buffer size calculation to be able to place all input fields without buffer overflow + meta-data.

### Why are the changes needed?
To make the test suite `MutableProjectionSuite` more stable.

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

### How was this patch tested?
By running the affected test suite:
```
$ build/sbt "test:testOnly *MutableProjectionSuite"
```

Closes #32339 from MaxGekk/fix-buffer-overflow-MutableProjectionSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-26 09:35:24 +03:00
Angerszhuuuu 6f782efb04 [SPARK-35220][SQL] DayTimeIntervalType/YearMonthIntervalType show different between Hive SerDe and row format delimited
### What changes were proposed in this pull request?
DayTimeIntervalType/YearMonthIntervalString show different between Hive SerDe and row format delimited.
Create this pr to add a test and  have disscuss.

For this problem I think we have two direction:

1. leave it as current and add a item t explain this  in migration guide docs.
2. Since we should not change hive serde's behavior, so we can cast spark row format delimited's behavior to use cast  DayTimeIntervalType/YearMonthIntervalType as HIVE_STYLE

### Why are the changes needed?
Add UT

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

### How was this patch tested?
added ut

Closes #32335 from AngersZhuuuu/SPARK-35220.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-26 11:26:32 +09:00
Kent Yao 5b1353f690 [SPARK-35168][SQL] mapred.reduce.tasks should be shuffle.partitions not adaptive.coalescePartitions.initialPartitionNum
### What changes were proposed in this pull request?

```sql
spark-sql> set spark.sql.adaptive.coalescePartitions.initialPartitionNum=1;
spark.sql.adaptive.coalescePartitions.initialPartitionNum	1
Time taken: 2.18 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks;
21/04/21 14:27:11 WARN SetCommand: Property mapred.reduce.tasks is deprecated, showing spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	1
Time taken: 0.03 seconds, Fetched 1 row(s)
spark-sql> set spark.sql.shuffle.partitions;
spark.sql.shuffle.partitions	200
Time taken: 0.024 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks=2;
21/04/21 14:31:52 WARN SetCommand: Property mapred.reduce.tasks is deprecated, automatically converted to spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	2
Time taken: 0.017 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks;
21/04/21 14:31:55 WARN SetCommand: Property mapred.reduce.tasks is deprecated, showing spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	1
Time taken: 0.017 seconds, Fetched 1 row(s)
spark-sql>
```

`mapred.reduce.tasks` is mapping to `spark.sql.shuffle.partitions` at write-side, but `spark.sql.adaptive.coalescePartitions.initialPartitionNum` might take precede of `spark.sql.shuffle.partitions`

### Why are the changes needed?

roundtrip for `mapred.reduce.tasks`

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

yes, `mapred.reduce.tasks` will always report `spark.sql.shuffle.partitions` whether `spark.sql.adaptive.coalescePartitions.initialPartitionNum` exists or not.

### How was this patch tested?

a new test

Closes #32265 from yaooqinn/SPARK-35168.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-25 20:27:12 +08:00
Maya Anderson 166cc6204c [SPARK-34990][SQL][TESTS] Add ParquetEncryptionSuite
### What changes were proposed in this pull request?

A simple test that writes and reads an encrypted parquet and verifies that it's encrypted by checking its magic string (in encrypted footer mode).

### Why are the changes needed?

To provide a test coverage for Parquet encryption.

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

No.

### How was this patch tested?

- [x] [SBT / Hadoop 3.2 / Java8 (the default)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137785/testReport)
- [ ] ~SBT / Hadoop 3.2 / Java11 by adding [test-java11] to the PR title.~ (Jenkins Java11 build is broken due to missing JDK11 installation)
- [x] [SBT / Hadoop 2.7 / Java8 by adding [test-hadoop2.7] to the PR title.](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137836/testReport)
- [x] Maven / Hadoop 3.2 / Java8 by adding [test-maven] to the PR title.
- [x] Maven / Hadoop 2.7 / Java8 by adding [test-maven][test-hadoop2.7] to the PR title.

Closes #32146 from andersonm-ibm/pme_testing.

Authored-by: Maya Anderson <mayaa@il.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-24 14:28:00 -07:00
Liang-Chi Hsieh b2a2b5d820 [SPARK-34297][SQL][SS] Add metrics for data loss and offset out range for KafkaMicroBatchStream
### What changes were proposed in this pull request?

This patch proposes to add a couple of metrics in scan node for Kafka batch streaming query.

### Why are the changes needed?

When testing SS, I found it is hard to track data loss of SS reading from Kafka. The micro batch scan node has only one metric, number of output rows. Users have no idea how many offsets to fetch are out of Kafka, how many times data loss happens. These metrics are important for users to know the quality of SS query running.

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

Yes, adding two metrics to micro batch scan node for Kafka batch streaming.

### How was this patch tested?

Currently I tested on internal cluster with Kafka:

<img width="1193" alt="Screen Shot 2021-04-22 at 7 16 29 PM" src="https://user-images.githubusercontent.com/68855/115808460-61bf8100-a39f-11eb-99a9-65d22c3f5fb0.png">

I was trying to add unit test. But as our batch streaming query disallows to specify ending offsets. If I only specify an out-of-range starting offset, when we get offset range in `getRanges`,  any negative size range will be filtered out. So it cannot actually test the case of fetched non-existing offset.

Closes #31398 from viirya/micro-batch-metrics.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-23 13:56:53 -07:00
Wenchen Fan a9345a0591 [SPARK-35204][SQL] CatalystTypeConverters of date/timestamp should accept both the old and new Java time classes
### What changes were proposed in this pull request?

`CatalystTypeConverters` is useful when the type of the input data classes are not known statically (otherwise we can use `ExpressionEncoder`). However, the current `CatalystTypeConverters` requires you to know the datetime data class statically, which makes it hard to use.

This PR improves the `CatalystTypeConverters` for date/timestamp, to support the old and new Java time classes at the same time.

### Why are the changes needed?

Make `CatalystTypeConverters` easier to use.

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

No.

### How was this patch tested?

new test

Closes #32312 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-23 19:21:43 +03:00
Angerszhuuuu e503b9c462 [SPARK-35201][SQL] Format empty grouping set exception in CUBE/ROLLUP
### What changes were proposed in this pull request?
Format empty grouping set exception in CUBE/ROLLUP

### Why are the changes needed?
Format empty grouping set exception in CUBE/ROLLUP

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

### How was this patch tested?
Not need

Closes #32307 from AngersZhuuuu/SPARK-35201.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-23 21:23:28 +09:00
Yingyi Bu 9af338cd68 [SPARK-35078][SQL] Add tree traversal pruning in expression rules
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- AND_OR
- BINARY_ARITHMETIC
- BINARY_COMPARISON
- CASE_WHEN
- CAST
- CONCAT
- COUNT
- IF
- LIKE_FAMLIY
- NOT
- NULL_CHECK
- UNARY_POSITIVE
- UPPER_OR_LOWER

Used them in the following rules:
- ConstantPropagation
- ReorderAssociativeOperator
- BooleanSimplification
- SimplifyBinaryComparison
- SimplifyCaseConversionExpressions
- SimplifyConditionals
- PushFoldableIntoBranches
- LikeSimplification
- NullPropagation
- SimplifyCasts
- RemoveDispensableExpressions
- CombineConcats

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32280 from sigmod/expression.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-23 16:33:58 +08:00
Wenchen Fan fdccd88c2a Revert "[SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function"
This reverts commit c8d78a70b4.
2021-04-23 15:55:30 +08:00
Cheng Su cab205e9e4 [SPARK-35141][SQL] Support two level of hash maps for final hash aggregation
### What changes were proposed in this pull request?

For partial hash aggregation (code-gen path), we have two level of hash map for aggregation. First level is from `RowBasedHashMapGenerator`, which is computation faster compared to the second level from `UnsafeFixedWidthAggregationMap`. The introducing of two level hash map can help improve CPU performance of query as the first level hash map normally fits in hardware cache and has cheaper hash function for key lookup.

For final hash aggregation, we can also support two level of hash map, to improve query performance further.
The original two level of hash map code works for final aggregation mostly out of box. The major change here is to support testing fall back of final aggregation (see change related to `bitMaxCapacity` and `checkFallbackForGeneratedHashMap`).

Example:

An aggregation query:

```
spark.sql(
  """
    |SELECT key, avg(value)
    |FROM agg1
    |GROUP BY key
  """.stripMargin)
```

The generated code for final aggregation is [here](https://gist.github.com/c21/20c10cc8e2c7e561aafbe9b8da055242).

An aggregation query with testing fallback:
```
withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> "2, 3") {
  spark.sql(
    """
      |SELECT key, avg(value)
      |FROM agg1
      |GROUP BY key
    """.stripMargin)
}
```
The generated code for final aggregation is [here](https://gist.github.com/c21/dabf176cbc18a5e2138bc0a29e81c878). Note the no more counter condition for first level fast map.

### Why are the changes needed?

Improve the CPU performance of hash aggregation query in general.

For `AggregateBenchmark."Aggregate w multiple keys"`, seeing query performance improved by 10%.
`codegen = T` means whole stage code-gen is enabled.
`hashmap = T` means two level maps is enabled for partial aggregation.
`finalhashmap = T` means two level maps is enabled for final aggregation.

```
Running benchmark: Aggregate w multiple keys
  Running case: codegen = F
  Stopped after 2 iterations, 8284 ms
  Running case: codegen = T hashmap = F
  Stopped after 2 iterations, 5424 ms
  Running case: codegen = T hashmap = T finalhashmap = F
  Stopped after 2 iterations, 4753 ms
  Running case: codegen = T hashmap = T finalhashmap = T
  Stopped after 2 iterations, 4508 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Aggregate w multiple keys:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
codegen = F                                        3881           4142         370          5.4         185.1       1.0X
codegen = T hashmap = F                            2701           2712          16          7.8         128.8       1.4X
codegen = T hashmap = T finalhashmap = F           2363           2377          19          8.9         112.7       1.6X
codegen = T hashmap = T finalhashmap = T           2252           2254           3          9.3         107.4       1.7X
```

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

No.

### How was this patch tested?

Existing unit test in `HashAggregationQuerySuite` and `HashAggregationQueryWithControlledFallbackSuite` already cover the test.

Closes #32242 from c21/agg.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-23 05:24:19 +00:00
Chao Sun 86238d0e88 [SPARK-35195][SQL][TEST] Move InMemoryTable etc to org.apache.spark.sql.connector.catalog
### What changes were proposed in this pull request?

Move the following classes:
- `InMemoryAtomicPartitionTable`
- `InMemoryPartitionTable`
- `InMemoryPartitionTableCatalog`
- `InMemoryTable`
- `InMemoryTableCatalog`
- `StagingInMemoryTableCatalog`

from `org.apache.spark.sql.connector` to `org.apache.spark.sql.connector.catalog`.

### Why are the changes needed?

These classes implement catalog related interfaces but reside in `org.apache.spark.sql.connector`. A more suitable place should be `org.apache.spark.sql.connector.catalog`.

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

No

### How was this patch tested?

N/A

Closes #32302 from sunchao/SPARK-35195.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-22 21:56:45 -07:00
Yingyi Bu 47f86875f7 [SPARK-35075][SQL] Add traversal pruning for subquery related rules
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- DYNAMIC_PRUNING_SUBQUERY
- EXISTS_SUBQUERY
- IN_SUBQUERY
- LIST_SUBQUERY
- PLAN_EXPRESSION
- SCALAR_SUBQUERY
- FILTER

Used them in the following rules:
- ResolveSubquery
- UpdateOuterReferences
- OptimizeSubqueries
- RewritePredicateSubquery
- PullupCorrelatedPredicates
- RewriteCorrelatedScalarSubquery (not the rule itself but an internal transform call, the full support is in SPARK-35148)
- InsertAdaptiveSparkPlan
- PlanAdaptiveSubqueries

### Why are the changes needed?

Reduce the number of tree traversals and hence improve the query compilation latency.

### How was this patch tested?

Existing tests.

Closes #32247 from sigmod/subquery.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-23 12:42:55 +08:00
Angerszhuuuu 04e2305a9b [SPARK-35187][SQL] Fix failure on the minimal interval literal
### What changes were proposed in this pull request?
If the sign '-' inside of interval string, everything is fine after bb5459fb26:
```
spark-sql> SELECT INTERVAL '-178956970-8' YEAR TO MONTH;
-178956970-8
```
but the sign outside of interval string is not handled properly:
```
spark-sql> SELECT INTERVAL -'178956970-8' YEAR TO MONTH;
Error in query:
Error parsing interval year-month string: integer overflow(line 1, pos 16)

== SQL ==
SELECT INTERVAL -'178956970-8' YEAR TO MONTH
----------------^^^
```
This pr fix this issue

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32296 from AngersZhuuuu/SPARK-35187.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-22 21:58:57 +03:00
beliefer 6c587d2627 [SPARK-35110][SQL] Handle ANSI intervals in WindowExecBase
### What changes were proposed in this pull request?
This PR makes window frame could support `YearMonthIntervalType` and `DayTimeIntervalType`.

### Why are the changes needed?
Extend the function of window frame

### Does this PR introduce _any_ user-facing change?
Yes. Users could use `YearMonthIntervalType` or `DayTimeIntervalType` as the sort expression for window frame.

### How was this patch tested?
New tests

Closes #32294 from beliefer/SPARK-35110.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-22 17:14:11 +03:00
Yingyi Bu 7f7a3d80a6 [SPARK-35183][SQL] Use transformAllExpressions in CombineConcats
### What changes were proposed in this pull request?

Use transformAllExpressions instead of transformExpressionsDown in CombineConcats. The latter only transforms the root plan node.

### Why are the changes needed?

It allows CombineConcats to cover more cases where `concat` are not in the root plan node.

### How was this patch tested?

Unit test. The updated tests would fail without the code change.

Closes #32290 from sigmod/concat.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-22 13:10:39 +00:00
Angerszhuuuu b22d54a58a [SPARK-35026][SQL] Support nested CUBE/ROLLUP/GROUPING SETS in GROUPING SETS
### What changes were proposed in this pull request?
PG and Oracle both support use CUBE/ROLLUP/GROUPING SETS in GROUPING SETS's grouping set as a sugar syntax.
![image](https://user-images.githubusercontent.com/46485123/114975588-139a1180-9eb7-11eb-8f53-498c1db934e0.png)

In this PR, we support it in Spark SQL too

### Why are the changes needed?
Keep consistent with PG and oracle

### Does this PR introduce _any_ user-facing change?
User can write grouping analytics like
```
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b));
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ());
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()));
```

### How was this patch tested?
Added Test

Closes #32201 from AngersZhuuuu/SPARK-35026.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-22 13:08:22 +00:00
Angerszhuuuu bb5459fb26 [SPARK-35177][SQL] Fix arithmetic overflow in parsing the minimal interval by IntervalUtils.fromYearMonthString
### What changes were proposed in this pull request?
IntervalUtils.fromYearMonthString should handle Int.MinValue months correctly.
In current logic, just use `Math.addExact(Math.multiplyExact(years, 12), months)` to calculate  negative total months will overflow when actual total months is Int.MinValue, this pr fixes this bug.

### Why are the changes needed?
IntervalUtils.fromYearMonthString should handle Int.MinValue months correctly

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

### How was this patch tested?
Added UT

Closes #32281 from AngersZhuuuu/SPARK-35177.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-22 11:59:38 +03:00
Liang-Chi Hsieh 548e66c98a [SPARK-34692][SQL][FOLLOWUP] Add INSET to ReplaceNullWithFalseInPredicate's pattern
### What changes were proposed in this pull request?

The test added by #31797 has the [failure](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137741/testReport/org.apache.spark.sql.catalyst.optimizer/ReplaceNullWithFalseInPredicateSuite/SPARK_34692__Support_Not_Int__and_Not_InSet__propagate_null/). This is a followup to fix it.

### Why are the changes needed?

Due to #32157, the rule `ReplaceNullWithFalseInPredicate` will check tree pattern before actually doing transformation. As `null` in `INSET` is not `NULL_LITERAL` pattern, we miss it and fail the newly added `not inset ...` check in `replaceNullWithFalse`.

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

No

### How was this patch tested?

Existing unit tests.

Closes #32278 from viirya/SPARK-34692-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-21 19:06:16 -07:00
Yuming Wang e609395913 [SPARK-34897][SQL] Support reconcile schemas based on index after nested column pruning
### What changes were proposed in this pull request?

It will remove `StructField` when [pruning nested columns](0f2c0b53e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala (L28-L42)). For example:
```scala
spark.sql(
  """
    |CREATE TABLE t1 (
    |  _col0 INT,
    |  _col1 STRING,
    |  _col2 STRUCT<c1: STRING, c2: STRING, c3: STRING, c4: BIGINT>)
    |USING ORC
    |""".stripMargin)

spark.sql("INSERT INTO t1 values(1, '2', struct('a', 'b', 'c', 10L))")

spark.sql("SELECT _col0, _col2.c1 FROM t1").show
```

Before this pr. The returned schema is: ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` add it will throw exception:
```
java.lang.AssertionError: assertion failed: The given data schema struct<_col0:int,_col2:struct<c1:string>> has less fields than the actual ORC physical schema, no idea which columns were dropped, fail to read.
	at scala.Predef$.assert(Predef.scala:223)
	at org.apache.spark.sql.execution.datasources.orc.OrcUtils$.requestedColumnIds(OrcUtils.scala:160)
```

After this pr. The returned schema is: ``` `_col0` INT,`_col1` STRING,`_col2` STRUCT<`c1`: STRING> ```.

The finally schema is ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` after the complete column pruning:
7a5647a93a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala (L208-L213)

e64eb75aed/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala (L96-L97)

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test.

Closes #31993 from wangyum/SPARK-34897.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-21 10:23:38 -07:00
ulysses-you 81dbaedede [SPARK-34692][SQL] Support Not(Int) and Not(InSet) propagate null in predicate
### What changes were proposed in this pull request?

* Add `Not(In)` and `Not(InSet)` check in `NullPropagation` rule.
* Add more test for `In` and `Not(In)` in `Project` level.

### Why are the changes needed?

The semantics of `Not(In)` could be seen like `And(a != b, a != c)` that match the `NullIntolerant`.

As we already simplify the `NullIntolerant` expression to null if it's children have null. E.g. `a != null` => `null`. It's safe to do this with `Not(In)`/`Not(InSet)`.

Note that, we can only do the simplify in predicate which `ReplaceNullWithFalseInPredicate`  rule do.

Let's say we have two sqls:
```
select 1 not in (2, null);
select 1 where 1 not in (2, null);
```
The first sql we cannot optimize since it would return `NULL` instead of `false`. The second is postive.

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

No.

### How was this patch tested?

Add test.

Closes #31797 from ulysses-you/SPARK-34692.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-21 15:37:02 +00:00
Gengliang Wang 43ad939a7e [SPARK-35152][SQL] ANSI mode: IntegralDivide throws exception on overflow
### What changes were proposed in this pull request?

IntegralDivide should throw an exception on overflow in ANSI mode.
There is only one case that can cause that:
```
Long.MinValue div -1
```

### Why are the changes needed?

ANSI compliance

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

Yes, IntegralDivide throws an exception on overflow in ANSI mode

### How was this patch tested?

Unit test

Closes #32260 from gengliangwang/integralDiv.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-21 16:08:20 +08:00
sandeep.katta 4f309cec07 [SPARK-35096][SQL] SchemaPruning should adhere spark.sql.caseSensitive config
### What changes were proposed in this pull request?

As a part of the SPARK-26837 pruning of nested fields from object serializers are supported. But it is missed to handle case insensitivity nature of spark

In this PR I have resolved the column names to be pruned based on `spark.sql.caseSensitive ` config
**Exception Before Fix**

```
Caused by: java.lang.ArrayIndexOutOfBoundsException: 0
  at org.apache.spark.sql.types.StructType.apply(StructType.scala:414)
  at org.apache.spark.sql.catalyst.optimizer.ObjectSerializerPruning$$anonfun$apply$4.$anonfun$applyOrElse$3(objects.scala:216)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.immutable.List.foreach(List.scala:392)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.immutable.List.map(List.scala:298)
  at org.apache.spark.sql.catalyst.optimizer.ObjectSerializerPruning$$anonfun$apply$4.applyOrElse(objects.scala:215)
  at org.apache.spark.sql.catalyst.optimizer.ObjectSerializerPruning$$anonfun$apply$4.applyOrElse(objects.scala:203)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:309)
  at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
  at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:309)
  at
```

### Why are the changes needed?
After Upgrade to Spark 3 `foreachBatch` API throws` java.lang.ArrayIndexOutOfBoundsException`. This issue will be fixed using this PR

### Does this PR introduce _any_ user-facing change?
No, Infact fixes the regression

### How was this patch tested?
Added tests and also tested verified manually

Closes #32194 from sandeep-katta/SPARK-35096.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-21 15:16:17 +08:00
Angerszhuuuu d259f9301a [SPARK-35113][SQL] Support ANSI intervals in the Hash expression
### What changes were proposed in this pull request?
Support ANSI interval in HashExpression and add UT

### Why are the changes needed?
Support ANSI interval in HashExpression

### Does this PR introduce _any_ user-facing change?
User can pass ANSI interval in HashExpression function

### How was this patch tested?
Added UT

Closes #32259 from AngersZhuuuu/SPARK-35113.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-21 07:44:44 +03:00
Kent Yao 81c3cc2312 [SPARK-35044][SQL][FOLLOWUP][TEST-HADOOP2.7] Fix hadoop 2.7 test due to diff between hadoop 2.7 and hadoop 3
### What changes were proposed in this pull request?

dfs.replication is inconsistent from hadoop 2.x to 3.x, so in this PR we use `dfs.hosts` to verify per https://github.com/apache/spark/pull/32144#discussion_r616833099

```
== Results ==
!== Correct Answer - 1 ==        == Spark Answer - 1 ==
!struct<>                        struct<key:string,value:string>
![dfs.replication,<undefined>]   [dfs.replication,3]
```

### Why are the changes needed?

fix Jenkins job with Hadoop 2.7

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

test only change
### How was this patch tested?

test only change

Closes #32263 from yaooqinn/SPARK-35044-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-21 09:10:05 +09:00
Max Gekk e8d6992b66 [SPARK-35153][SQL] Make textual representation of ANSI interval operators more readable
### What changes were proposed in this pull request?
In the PR, I propose to override the `sql` and `toString` methods of the expressions that implement operators over ANSI intervals (`YearMonthIntervalType`/`DayTimeIntervalType`), and replace internal expression class names by operators like `*`, `/` and `-`.

### Why are the changes needed?
Proposed methods should make the textual representation of such operators more readable, and potentially parsable by Spark SQL parser.

### Does this PR introduce _any_ user-facing change?
Yes. This can influence on column names.

### How was this patch tested?
By running existing test suites for interval and datetime expressions, and re-generating the `*.sql` tests:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
```

Closes #32262 from MaxGekk/interval-operator-sql.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-20 23:13:40 +03:00
Wenchen Fan e08c40fa3f [SPARK-35145][SQL] CurrentOrigin should support nested invoking
### What changes were proposed in this pull request?

`CurrentOrigin` is a thread-local variable to track the original SQL line position in plan/expression. Usually, we set `CurrentOrigin`, create `TreeNode` instances, and reset `CurrentOrigin`.

This PR updates the last step to set `CurrentOrigin` to its previous value, instead of resetting it. This is necessary when we invoke `CurrentOrigin` in a nested way, like with subqueries.

### Why are the changes needed?

To keep the original SQL line position in the error message in more cases.

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

No, only minor error message changes.

### How was this patch tested?

existing tests

Closes #32249 from cloud-fan/origin.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 15:23:41 +00:00
Liang-Chi Hsieh eb9a4390da [SPARK-34338][SQL] Report metrics from Datasource v2 scan
### What changes were proposed in this pull request?

This patch proposes to leverage `CustomMetric`, `CustomTaskMetric` API to report custom metrics from DS v2 scan to Spark.

### Why are the changes needed?

This is related to #31398. In SPARK-34297, we want to add a couple of metrics when reading from Kafka in SS. We need some public API change in DS v2 to make it possible. This extracts only DS v2 change and make it general for DS v2 instead of micro-batch DS v2 API.

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

No

### How was this patch tested?

Unit test.

Implement a simple test DS v2 class locally and run it:

```scala
scala> import org.apache.spark.sql.execution.datasources.v2._
import org.apache.spark.sql.execution.datasources.v2._

scala> classOf[CustomMetricDataSourceV2].getName
res0: String = org.apache.spark.sql.execution.datasources.v2.CustomMetricDataSourceV2

scala> val df = spark.read.format(res0).load()
df: org.apache.spark.sql.DataFrame = [i: int, j: int]

scala> df.collect
```

<img width="703" alt="Screen Shot 2021-03-30 at 11 07 13 PM" src="https://user-images.githubusercontent.com/68855/113098080-d8a49800-91ac-11eb-8681-be408a0f2e69.png">

Closes #31451 from viirya/dsv2-metrics.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 15:01:44 +00:00
Angerszhuuuu 361444890e [SPARK-34035][SQL] Refactor ScriptTransformation to remove input parameter and replace it by child.output
### What changes were proposed in this pull request?
Refactor ScriptTransformation to remove input parameter and replace it by child.output

### Why are the changes needed?
refactor code

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

### How was this patch tested?
Existed UT

Closes #32228 from AngersZhuuuu/SPARK-34035.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 14:52:21 +00:00
Angerszhuuuu b219e37af3 [SPARK-35068][SQL] Add tests for ANSI intervals to HiveThriftBinaryServerSuite
### What changes were proposed in this pull request?
After the PR https://github.com/apache/spark/pull/32209, this should be possible now.
We can add test case for ANSI intervals to HiveThriftBinaryServerSuite

### Why are the changes needed?
Add more test case

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

### How was this patch tested?
Added UT

Closes #32250 from AngersZhuuuu/SPARK-35068.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-20 13:17:59 +03:00
allisonwang-db b6bb24ca1b [SPARK-34974][SQL] Improve subquery decorrelation framework
### What changes were proposed in this pull request?
This PR implements the decorrelation technique in the paper "Unnesting Arbitrary Queries" by T. Neumann; A. Kemper
(http://www.btw-2015.de/res/proceedings/Hauptband/Wiss/Neumann-Unnesting_Arbitrary_Querie.pdf). It currently supports Filter, Project, Aggregate, Join, and UnaryNode that passes CheckAnalysis.

This feature can be controlled by the config `spark.sql.optimizer.decorrelateInnerQuery.enabled` (default: true).

A few notes:
1. This PR does not relax any constraints in CheckAnalysis for correlated subqueries, even though some cases can be supported by this new framework, such as aggregate with correlated non-equality predicates. This PR focuses on adding the new framework and making sure all existing cases can be supported. Constraints can be relaxed gradually in the future via separate PRs.
2. The new framework is only enabled for correlated scalar subqueries, as the first step. EXISTS/IN subqueries can be supported in the future.

### Why are the changes needed?
Currently, Spark has limited support for correlated subqueries. It only allows `Filter` to reference outer query columns and does not support non-equality predicates when the subquery is aggregated. This new framework will allow more operators to host outer column references and support correlated non-equality predicates and more types of operators in correlated subqueries.

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

### How was this patch tested?
Existing unit and SQL query tests and new optimizer plan tests.

Closes #32072 from allisonwang-db/spark-34974-decorrelation.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 09:22:22 +00:00
Max Gekk aa0d00de5e [SPARK-35018][SQL][TESTS] Check transferring of year-month intervals via Hive Thrift server
### What changes were proposed in this pull request?
1. Add a test to check that Thrift server is able to collect year-month intervals and transfer them via thrift protocol.
2. Improve similar test for day-time intervals. After the changes, the test doesn't depend on the result of date subtractions. In the future, the type of date subtract can be changed. So, current PR should make the test tolerant to the changes.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the modified test suite:
```
$ ./build/sbt -Phive -Phive-thriftserver "test:testOnly *SparkThriftServerProtocolVersionsSuite"
```

Closes #32240 from MaxGekk/year-month-interval-thrift-protocol.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-20 08:52:37 +03:00
Yingyi Bu f4926d1c8b [SPARK-35052][SQL] Use static bits for AttributeReference and Literal
### What changes were proposed in this pull request?

- Share a static ImmutableBitSet for `treePatternBits` in all object instances of AttributeReference.
- Share three static ImmutableBitSets for  `treePatternBits` in three kinds of Literals.
- Add an ImmutableBitSet as a subclass of BitSet.

### Why are the changes needed?

Reduce the additional memory usage caused by `treePatternBits`.

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

No.

### How was this patch tested?

Existing tests.

Closes #32157 from sigmod/leaf.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-20 13:13:16 +08:00
allisonwang-db bad4b6f025 [SPARK-35080][SQL] Only allow a subset of correlated equality predicates when a subquery is aggregated
### What changes were proposed in this pull request?
This PR updated the `foundNonEqualCorrelatedPred` logic for correlated subqueries in `CheckAnalysis` to only allow correlated equality predicates that guarantee one-to-one mapping between inner and outer attributes, instead of all equality predicates.

### Why are the changes needed?
To fix correctness bugs. Before this fix Spark can give wrong results for certain correlated subqueries that pass CheckAnalysis:
Example 1:
```sql
create or replace view t1(c) as values ('a'), ('b')
create or replace view t2(c) as values ('ab'), ('abc'), ('bc')

select c, (select count(*) from t2 where t1.c = substring(t2.c, 1, 1)) from t1
```
Correct results: [(a, 2), (b, 1)]
Spark results:
```
+---+-----------------+
|c  |scalarsubquery(c)|
+---+-----------------+
|a  |1                |
|a  |1                |
|b  |1                |
+---+-----------------+
```
Example 2:
```sql
create or replace view t1(a, b) as values (0, 6), (1, 5), (2, 4), (3, 3);
create or replace view t2(c) as values (6);

select c, (select count(*) from t1 where a + b = c) from t2;
```
Correct results: [(6, 4)]
Spark results:
```
+---+-----------------+
|c  |scalarsubquery(c)|
+---+-----------------+
|6  |1                |
|6  |1                |
|6  |1                |
|6  |1                |
+---+-----------------+
```
### Does this PR introduce _any_ user-facing change?
Yes. Users will not be able to run queries that contain unsupported correlated equality predicates.

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

Closes #32179 from allisonwang-db/spark-35080-subquery-bug.

Lead-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 11:11:40 +08:00
Yingyi Bu 9a6d7730f5 [SPARK-35103][SQL] Make TypeCoercion rules more efficient
## What changes were proposed in this pull request?
This PR fixes a couple of things in TypeCoercion rules:
- Only run the propagate types step if the children of a node have output attributes with changed dataTypes and/or nullability. This is implemented as custom tree transformation. The TypeCoercion rules now only implement a partial function.
- Combine multiple type coercion rules into a single rule. Multiple rules are applied in single tree traversal.
- Reduce calls to conf.get in DecimalPrecision. This now happens once per tree traversal, instead of once per matched expression.
- Reduce the use of withNewChildren.

This brings down the number of CPU cycles spend in analysis by ~28% (benchmark: 10 iterations of all TPC-DS queries on SF10).

## How was this patch tested?
Existing tests.

Closes #32208 from sigmod/coercion.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-19 21:25:58 +02:00
Cheng Pan 0c2e9b99aa [SPARK-35138][SQL] Remove Antlr4 workaround
### What changes were proposed in this pull request?

Remove Antlr 4.7 workaround.

### Why are the changes needed?

The https://github.com/antlr/antlr4/commit/ac9f7530 has been fixed in upstream, so remove the workaround to simplify code.

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

No

### How was this patch tested?

Existed UTs.

Closes #32238 from pan3793/antlr-minor.

Authored-by: Cheng Pan <379377944@qq.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-19 10:38:53 -07:00
Kent Yao 2d161cb3a1 [SPARK-35102][SQL] Make spark.sql.hive.version read-only, not deprecated and meaningful
### What changes were proposed in this pull request?

Firstly let's take a look at the definition and comment.

```
// A fake config which is only here for backward compatibility reasons. This config has no effect
// to Spark, just for reporting the builtin Hive version of Spark to existing applications that
// already rely on this config.
val FAKE_HIVE_VERSION = buildConf("spark.sql.hive.version")
  .doc(s"deprecated, please use ${HIVE_METASTORE_VERSION.key} to get the Hive version in Spark.")
  .version("1.1.1")
  .fallbackConf(HIVE_METASTORE_VERSION)
```
It is used for reporting the built-in Hive version but the current status is unsatisfactory, as it is could be changed in many ways e.g. --conf/SET syntax.

It is marked as deprecated but kept a long way until now. I guess it is hard for us to remove it and not even necessary.

On second thought, it's actually good for us to keep it to work with the `spark.sql.hive.metastore.version`. As when `spark.sql.hive.metastore.version` is changed, it could be used to report the compiled hive version statically, it's useful when an error occurs in this case. So this parameter should be fixed to compiled hive version.

### Why are the changes needed?

`spark.sql.hive.version` is useful in certain cases and should be read-only

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

`spark.sql.hive.version` now is read-only

### How was this patch tested?

new test cases

Closes #32200 from yaooqinn/SPARK-35102.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 14:40:21 +00:00
gengjiaan 7f3403583f [SPARK-34715][SQL][TESTS] Add round trip tests for period <-> month and duration <-> micros
### What changes were proposed in this pull request?
Similarly to the test from the PR https://github.com/apache/spark/pull/31799, add tests:
1. Months -> Period -> Months
2. Period -> Months -> Period
3. Duration -> micros -> Duration

### Why are the changes needed?
Add round trip tests for period <-> month and duration <-> micros

### Does this PR introduce _any_ user-facing change?
'No'. Just test cases.

### How was this patch tested?
Jenkins test

Closes #32234 from beliefer/SPARK-34715.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 16:13:13 +03:00
Max Gekk 1d1ed3eb25 [SPARK-35107][SQL] Parse unit-to-unit interval literals to ANSI intervals
### What changes were proposed in this pull request?
Parse the year-month interval literals like `INTERVAL '1-1' YEAR TO MONTH` to values of `YearMonthIntervalType`, and day-time interval literals to `DayTimeIntervalType` values. Currently, Spark SQL supports:
- DAY TO HOUR
- DAY TO MINUTE
- DAY TO SECOND
- HOUR TO MINUTE
- HOUR TO SECOND
- MINUTE TO SECOND

All such interval literals are converted to `DayTimeIntervalType`, and `YEAR TO MONTH` to `YearMonthIntervalType` while loosing info about `from` and `to` units.

**Note**: new behavior is under the SQL config `spark.sql.legacy.interval.enabled` which is `false` by default. When the config is set to `true`, the interval literals are parsed to `CaledarIntervalType` values.

Closes #32176

### Why are the changes needed?
To conform the ANSI SQL standard which assumes conversions of interval literals to year-month or day-time interval but not to mixed interval type like Catalyst's `CalendarIntervalType`.

### Does this PR introduce _any_ user-facing change?
Yes.

Before:
```sql
spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND;
1 days 1 hours 2 minutes 3.123 seconds
spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND);
interval
```

After:
```sql
spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND;
1 01:02:03.123000000
spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND);
day-time interval
```

### How was this patch tested?
1. By running the affected test suites:
```
$ ./build/sbt "test:testOnly *.ExpressionParserSuite"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z create_view.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
```
2. PostgresSQL tests are executed with `spark.sql.legacy.interval.enabled` is set to `true` to keep compatibility with PostgreSQL output:
```sql
> SELECT interval '999' second;
0 years 0 mons 0 days 0 hours 16 mins 39.00 secs
```

Closes #32209 from MaxGekk/parse-ansi-interval-literals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 16:00:59 +03:00
gengjiaan 8dc455bba8 [SPARK-34837][SQL] Support ANSI SQL intervals by the aggregate function avg
### What changes were proposed in this pull request?
Extend the `Average` expression to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the overflow exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
Extend `org.apache.spark.sql.catalyst.expressions.aggregate.Average` to support `DayTimeIntervalType` and `YearMonthIntervalType`.

### Does this PR introduce _any_ user-facing change?
'No'.
Should not since new types have not been released yet.

### How was this patch tested?
Jenkins test

Closes #32229 from beliefer/SPARK-34837.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 15:56:56 +03:00
HyukjinKwon 70b606ffdd [SPARK-35045][SQL][FOLLOW-UP] Add a configuration for CSV input buffer size
### What changes were proposed in this pull request?

This PR makes the input buffer configurable (as an internal configuration). This is mainly to work around the regression in uniVocity/univocity-parsers#449.

This is particularly useful for SQL workloads that requires to rewrite the `CREATE TABLE` with options.

### Why are the changes needed?

To work around uniVocity/univocity-parsers#449.

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

No, it's only internal option.

### How was this patch tested?

Manually tested by modifying the unittest added in https://github.com/apache/spark/pull/31858 as below:

```diff
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index fd25a79619d..705f38dbfbd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
 -2456,6 +2456,7  abstract class CSVSuite
   test("SPARK-34768: counting a long record with ignoreTrailingWhiteSpace set to true") {
     val bufSize = 128
     val line = "X" * (bufSize - 1) + "| |"
+    spark.conf.set("spark.sql.csv.parser.inputBufferSize", 128)
     withTempPath { path =>
       Seq(line).toDF.write.text(path.getAbsolutePath)
       assert(spark.read.format("csv")
```

Closes #32231 from HyukjinKwon/SPARK-35045-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-19 19:52:06 +09:00
Angerszhuuuu a74f601040 [SPARK-31937][SQL] Support processing ArrayType/MapType/StructType data using no-serde mode script transform
### What changes were proposed in this pull request?
Support no-serde mode script transform use ArrayType/MapType/StructStpe data.

### Why are the changes needed?
Make user can process array/map/struct data

### Does this PR introduce _any_ user-facing change?
Yes, user can process array/map/struct data in script transform `no-serde` mode

### How was this patch tested?
Added UT

Closes #30957 from AngersZhuuuu/SPARK-31937.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-19 17:02:32 +09:00
Terry Kim 7a06cdd53b [SPARK-35122][SQL] Migrate CACHE/UNCACHE TABLE to use AnalysisOnlyCommand
### What changes were proposed in this pull request?

Now that `AnalysisOnlyCommand` in introduced in #32032, `CacheTable` and `UncacheTable` can extend `AnalysisOnlyCommand` to simplify the code base. For example, the logic to handle these commands such that the tables are only analyzed is scattered across different places.

### Why are the changes needed?

To simplify the code base to handle these two commands.

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

No, just internal refactoring.

### How was this patch tested?

The existing tests (e.g., `CachedTableSuite`) cover the changes in this PR. For example, if I make `CacheTable`/`UncacheTable` extend `LeafCommand`, there are few failures in `CachedTableSuite`.

Closes #32220 from imback82/cache_cmd_analysis_only.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 06:00:23 +00:00
Peter Toth c8d78a70b4 [SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function
### What changes were proposed in this pull request?
This PR:
- Adds a new expression `GroupingExprRef` that can be used in aggregate expressions of `Aggregate` nodes to refer grouping expressions by index. These expressions capture the data type and nullability of the referred grouping expression.
- Adds a new rule `EnforceGroupingReferencesInAggregates` that inserts the references in the beginning of the optimization phase.
- Adds a new rule `UpdateGroupingExprRefNullability` to update nullability of `GroupingExprRef` expressions as nullability of referred grouping expression can change during optimization.

### Why are the changes needed?
If aggregate expressions (without aggregate functions) in an `Aggregate` node are complex then the `Optimizer` can optimize out grouping expressions from them and so making aggregate expressions invalid.

Here is a simple example:
```
SELECT not(t.id IS NULL) , count(*)
FROM t
GROUP BY t.id IS NULL
```
In this case the `BooleanSimplification` rule does this:
```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.BooleanSimplification ===
!Aggregate [isnull(id#222)], [NOT isnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]   Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#226, count(1) AS c#224L]
 +- Project [value#219 AS id#222]                                                                 +- Project [value#219 AS id#222]
    +- LocalRelation [value#219]                                                                     +- LocalRelation [value#219]
```
where `NOT isnull(id#222)` is optimized to `isnotnull(id#222)` and so it no longer refers to any grouping expression.

Before this PR:
```
== Optimized Logical Plan ==
Aggregate [isnull(id#222)], [isnotnull(id#222) AS (NOT (id IS NULL))#234, count(1) AS c#232L]
+- Project [value#219 AS id#222]
   +- LocalRelation [value#219]
```
and running the query throws an error:
```
Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
java.lang.IllegalStateException: Couldn't find id#222 in [isnull(id#222)#230,count(1)#226L]
```

After this PR:
```
== Optimized Logical Plan ==
Aggregate [isnull(id#222)], [NOT groupingexprref(0) AS (NOT (id IS NULL))#234, count(1) AS c#232L]
+- Project [value#219 AS id#222]
   +- LocalRelation [value#219]
```
and the query works.

### Does this PR introduce _any_ user-facing change?
Yes, the query works.

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

Closes #31913 from peter-toth/SPARK-34581-keep-grouping-expressions.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 04:58:41 +00:00
Cheng Su fd08c93151 [SPARK-35109][SQL] Fix minor exception messages of HashedRelation and HashJoin
### What changes were proposed in this pull request?

It seems that we miss classifying one `SparkOutOfMemoryError` in `HashedRelation`. Add the error classification for it. In addition, clean up two errors definition of `HashJoin` as they are not used.

### Why are the changes needed?

Better error classification.

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

No.

### How was this patch tested?

Existing tests.

Closes #32211 from c21/error-message.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-19 12:43:43 +09:00
Max Gekk 074f770137 [SPARK-35115][SQL][TESTS] Check ANSI intervals in MutableProjectionSuite
### What changes were proposed in this pull request?
Add checks for `YearMonthIntervalType` and `DayTimeIntervalType` to `MutableProjectionSuite`.

### Why are the changes needed?
To improve test coverage, and the same checks as for `CalendarIntervalType`.

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *MutableProjectionSuite"
```

Closes #32225 from MaxGekk/test-ansi-intervals-in-MutableProjectionSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-19 08:50:19 +09:00
gengjiaan 12abfe7917 [SPARK-34716][SQL] Support ANSI SQL intervals by the aggregate function sum
### What changes were proposed in this pull request?
Extend the `Sum` expression to  to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the overflow exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
Extend `org.apache.spark.sql.catalyst.expressions.aggregate.Sum` to support `DayTimeIntervalType` and `YearMonthIntervalType`.

### Does this PR introduce _any_ user-facing change?
'No'.
Should not since new types have not been released yet.

### How was this patch tested?
Jenkins test

Closes #32107 from beliefer/SPARK-34716.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-18 18:03:50 +03:00
Max Gekk d04b467690 [SPARK-35114][SQL][TESTS] Add checks for ANSI intervals to LiteralExpressionSuite
### What changes were proposed in this pull request?
In the PR, I propose to add additional checks for ANSI interval types `YearMonthIntervalType` and `DayTimeIntervalType` to `LiteralExpressionSuite`.

Also, I replaced some long literal values by `CalendarInterval` to check `CalendarIntervalType` that the tests were supposed to check.

### Why are the changes needed?
To improve test coverage and have the same checks for ANSI types as for `CalendarIntervalType`.

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *LiteralExpressionSuite"
```

Closes #32213 from MaxGekk/interval-literal-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-18 11:35:00 +03:00
beliefer 03191e8d8f [SPARK-35116][SQL][TESTS] The generated data fits the precision of DayTimeIntervalType in spark
### What changes were proposed in this pull request?
The precision of `java.time.Duration` is nanosecond, but when it is used as `DayTimeIntervalType` in Spark, it is microsecond.
At present, the `DayTimeIntervalType` data generated in the implementation of `RandomDataGenerator` is accurate to nanosecond, which will cause the `DayTimeIntervalType` to be converted to long, and then back to `DayTimeIntervalType` to lose the accuracy, which will cause the test to fail. For example: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137390/testReport/org.apache.spark.sql.hive.execution/HashAggregationQueryWithControlledFallbackSuite/udaf_with_all_data_types/

### Why are the changes needed?
Improve `RandomDataGenerator` so that the generated data fits the precision of DayTimeIntervalType in spark.

### Does this PR introduce _any_ user-facing change?
'No'. Just change the test class.

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

Closes #32212 from beliefer/SPARK-35116.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-18 09:04:11 +03:00
Kousuke Saruta 95db7e6459 [SPARK-35104][SQL] Fix ugly indentation of multiple JSON records in a single split file generated by JacksonGenerator when pretty option is true
### What changes were proposed in this pull request?

This issue fixes an issue that indentation of multiple output JSON records in a single split file are broken except for the first record in the split when `pretty` option is `true`.
```
// Run in the Spark Shell.
// Set spark.sql.leafNodeDefaultParallelism to 1 for the current master.
// Or set spark.default.parallelism for the previous releases.
spark.conf.set("spark.sql.leafNodeDefaultParallelism", 1)
val df = Seq("a", "b", "c").toDF
df.write.option("pretty", "true").json("/path/to/output")

# Run in a Shell
$ cat /path/to/output/*.json
{
  "value" : "a"
}
 {
  "value" : "b"
}
 {
  "value" : "c"
}
```

### Why are the changes needed?

It's not pretty even though `pretty` option is true.

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

I think "No". Indentation style is changed but JSON format is not changed.

### How was this patch tested?

New test.

Closes #32203 from sarutak/fix-ugly-indentation.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-16 11:00:52 +03:00
Max Gekk 3f4c32b3ca [SPARK-35099][SQL] Convert ANSI interval literals to SQL string in ANSI style
### What changes were proposed in this pull request?
Handle `YearMonthIntervalType` and `DayTimeIntervalType` in the `sql()` and `toString()` method of `Literal`, and format the ANSI interval in the ANSI style.

### Why are the changes needed?
To improve readability and UX with Spark SQL. For example, a test output before the changes:
```
-- !query
select timestamp'2011-11-11 11:11:11' - interval '2' day
-- !query schema
struct<TIMESTAMP '2011-11-11 11:11:11' - 172800000000:timestamp>
-- !query output
2011-11-09 11:11:11
```

### Does this PR introduce _any_ user-facing change?
Should not since the new intervals haven't been released yet.

### How was this patch tested?
By running new tests:
```
$ ./build/sbt "test:testOnly *LiteralExpressionSuite"
```

Closes #32196 from MaxGekk/literal-ansi-interval-sql.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-16 08:22:25 +03:00
Angerszhuuuu 71133e1c2a [SPARK-35070][SQL] TRANSFORM not support alias in inputs
### What changes were proposed in this pull request?
Normal function parameters should not support alias, hive not support too
![image](https://user-images.githubusercontent.com/46485123/114645556-4a7ff400-9d0c-11eb-91eb-bc679ea0039a.png)
In this pr we forbid use alias in `TRANSFORM`'s inputs

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32165 from AngersZhuuuu/SPARK-35070.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-15 06:51:07 +00:00
Angerszhuuuu 9b2e0d6191 [SPARK-35086][SQL][CORE] --verbose should be passed to Spark SQL CLI too
### What changes were proposed in this pull request?
In current code, if we run spark sql with
```
./bin/spark-sql --verbose
```
It won't be passed to end SparkSQLCliDriver, then the SessionState won't call `setIsVerbose`

In the CLI option, it shows
```
CLI options:
 -v,--verbose                     Verbose mode (echo executed SQL to the
                                  console)
```

It's not consistent. This pr fix this issue
### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
when user call `-v` when run spark sql, sql will be echoed to console.

### How was this patch tested?
Added UT

Closes #32163 from AngersZhuuuu/SPARK-35086.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-04-15 12:59:20 +08:00
Kousuke Saruta 271aa331b3 [MINOR][SQL] Refactor the comments in HiveClientImpl.withHiveState
### What changes were proposed in this pull request?

This PR refactors three parts of the comments in `HiveClientImpl.withHiveState`

One is about the following comment.
```
// The classloader in clientLoader could be changed after addJar, always use the latest
// classloader.
```
The comment was added in SPARK-10810 (#8909) because `IsolatedClientLoader.classLoader` was declared as `var`.
But the field is now `val` and cannot be changed after instanciation.
So, the comment can confuse developers.

One is about the following code and comment.
```
// classloader. We explicitly set the context class loader since "conf.setClassLoader" does
// not do that, and the Hive client libraries may need to load classes defined by the client's
// class loader.
Thread.currentThread().setContextClassLoader(clientLoader.classLoader)
```
It's not trivial why this part is necessary and it's difficult when we can remove this code in the future.
So, I revised the comment by adding the reference of the related JIRA.

And the last one is about the following code and comment.
```
// Replace conf in the thread local Hive with current conf
Hive.get(conf)
```
It's also not trivial why this part is necessary.
I revised the comment by adding the reference of the related discussion.

### Why are the changes needed?

To make code more readable.

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

No.

### How was this patch tested?

It's just a comment refactoring so I add no new test.

Closes #32162 from sarutak/refactor-HiveClientImpl.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-14 21:42:35 -07:00
Kent Yao f32114d17e [SPARK-35044][SQL] SET propertyKey shall also lookup sparkSession.sharedState.hadoopConf to display the effective default hive/hadoop configs
### What changes were proposed in this pull request?

Currently, pure SQL users are short of ways to see the Hadoop configurations which may affect their jobs a lot, they are only able to get the Hadoop configs that exist in `SQLConf` while other defaults in `SharedState.hadoopConf` display wrongly and confusingly with `<undefined>`.

The pre-loaded ones from `core-site.xml, hive-site.xml` etc., will only stay in `sparkSession.sharedState.hadoopConf` or `sc._hadoopConfiguation` not `SQLConf`. Some of them that related the Hive Metastore connection(never change it spark runtime), e.g. `hive.metastore.uris`, are clearly global static and unchangeable but displayable I guess. Some of the ones that might be related to, for example, the output codec/compression, preset in Hadoop/hive config files like core-site.xml shall be still changeable from case to case, table to table, file to file, etc. It' meaningfully to show the defaults for users to change based on that.

In this PR, I propose to support get a Hadoop configuration by SET syntax, for example
```
SET mapreduce.map.output.compress.codec;
```

### Why are the changes needed?

better user experience for pure SQL users

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

yes, where retrieving a conf only existing in sessionState.hadoopConf, before is `undefined` and now you see it

### How was this patch tested?

new test

Closes #32144 from yaooqinn/SPARK-35044.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-15 01:44:10 +08:00
Max Gekk de9e8b6c94 [SPARK-35051][SQL] Support add/subtract of a day-time interval to/from a date
### What changes were proposed in this pull request?
Support `date +/- day-time interval`. In the PR, I propose to update the binary arithmetic rules, and cast an input date to a timestamp at the session time zone, and then add a day-time interval to it.

### Why are the changes needed?
1. To conform the ANSI SQL standard which requires to support such operation over dates and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">
2. To fix the regression comparing to the recent Spark release 3.1 with default settings.

Before the changes:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
Error in query: cannot resolve 'DATE '2021-04-14' + subtracttimestamps(TIMESTAMP '2021-04-14 18:14:56.497', TIMESTAMP '2021-04-13 00:00:00')' due to data type mismatch: argument 1 requires timestamp type, however, 'DATE '2021-04-14'' is of date type.; line 1 pos 7;
'Project [unresolvedalias(cast(2021-04-14 + subtracttimestamps(2021-04-14 18:14:56.497, 2021-04-13 00:00:00, false, Some(Europe/Moscow)) as date), None)]
+- OneRowRelation
```

Spark 3.1:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
2021-04-15
```

Hive:
```sql
0: jdbc:hive2://localhost:10000/default> select date'2021-04-14' + (timestamp'2020-04-14 18:15:30' - timestamp'2020-04-13 00:00:00');
+------------------------+
|          _c0           |
+------------------------+
| 2021-04-15 18:15:30.0  |
+------------------------+
```

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

After the changes:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
2021-04-15 18:13:16.555
```

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #32170 from MaxGekk/date-add-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-14 19:28:26 +03:00
Angerszhuuuu 4ca9958270 [SPARK-35069][SQL] TRANSFORM forbidden DISTICNT and ALL, also make the error clear
### What changes were proposed in this pull request?
According to https://github.com/apache/spark/pull/29087#discussion_r612267050,  add UT in `transform.sql`

It seems that distinct is not recognized as a reserved word here

```
-- !query
explain extended SELECT TRANSFORM(distinct b, a, c)
                   USING 'cat' AS (a, b, c)
                 FROM script_trans
                 WHERE a <= 4
-- !query schema
struct<plan:string>
-- !query output
== Parsed Logical Plan ==
'ScriptTransformation [*], cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false)
+- 'Project ['distinct AS b#x, 'a, 'c]
   +- 'Filter ('a <= 4)
      +- 'UnresolvedRelation [script_trans], [], false

== Analyzed Logical Plan ==
org.apache.spark.sql.AnalysisException: cannot resolve 'distinct' given input columns: [script_trans.a, script_trans.b, script_trans.c]; line 1 pos 34;
'ScriptTransformation [*], cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false)
+- 'Project ['distinct AS b#x, a#x, c#x]
   +- Filter (a#x <= 4)
      +- SubqueryAlias script_trans
         +- View (`script_trans`, [a#x,b#x,c#x])
            +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x]
               +- Project [a#x, b#x, c#x]
                  +- SubqueryAlias script_trans
                     +- LocalRelation [a#x, b#x, c#x]
```

Hive's error
![image](https://user-images.githubusercontent.com/46485123/114533170-355d8380-9c80-11eb-992f-982f0b296759.png)

### Why are the changes needed?

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

### How was this patch tested?
Added Ut

Closes #32149 from AngersZhuuuu/SPARK-28227-new-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 15:03:29 +00:00
Terry Kim b5241c97b1 [SPARK-34701][SQL] Introduce AnalysisOnlyCommand that allows its children to be removed once the command is marked as analyzed
### What changes were proposed in this pull request?

This PR proposes to introduce the `AnalysisOnlyCommand` trait such that a command that extends this trait can have its children only analyzed, but not optimized. There is a corresponding analysis rule `HandleAnalysisOnlyCommand` that marks the command as analyzed after all other analysis rules are run.

This can be useful if a logical plan has children where they need to be only analyzed, but not optimized - e.g., `CREATE VIEW` or `CACHE TABLE AS`. This also addresses the issue found in #31933.

This PR also updates `CreateViewCommand`, `CacheTableAsSelect`, and `AlterViewAsCommand` to use the new trait / rule such that their children are only analyzed.

### Why are the changes needed?

To address the issue where the plan is unnecessarily re-analyzed in `CreateViewCommand`.

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

No.

### How was this patch tested?

Existing tests should cover the changes.

Closes #32032 from imback82/skip_transform.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 08:24:25 +00:00
Karen Feng 816f6dd13e [SPARK-34527][SQL] Resolve duplicated common columns from USING/NATURAL JOIN
### What changes were proposed in this pull request?

Adds the duplicated common columns as hidden columns to the Projection used to rewrite NATURAL/USING JOINs.

### Why are the changes needed?

Allows users to resolve either side of the NATURAL/USING JOIN's common keys.
Previously, the user could only resolve the following columns:

| Join type | Left key columns | Right key columns |
| --- | --- | --- |
| Inner | Yes | No |
| Left | Yes | No |
| Right | No | Yes |
| Outer | No | No |

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

Yes. The user can now symmetrically resolve the common columns from a NATURAL/USING JOIN.

### How was this patch tested?

SQL-side tests. The behavior matches PostgreSQL and MySQL.

Closes #31666 from karenfeng/spark-34527.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 07:01:40 +00:00
Kousuke Saruta ef05e89ee5 [SPARK-34977][SQL] LIST FILES/JARS/ARCHIVES cannot handle multiple arguments properly when at least one path is quoted
### What changes were proposed in this pull request?

This PR fixes an issue that `LIST FILES/JARS/ARCHIVES path1 path2 ...` cannot list all paths if at least one path is quoted.
An example here.
```
ADD FILE /tmp/test1;
ADD FILE /tmp/test2;

LIST FILES /tmp/test1 /tmp/test2;
file:/tmp/test1
file:/tmp/test2

LIST FILES /tmp/test1 "/tmp/test2";
file:/tmp/test2
```

In this example, the second `LIST FILES` doesn't show `file:/tmp/test1`.

To resolve this issue, I modified the syntax rule to be able to handle this case.
I also changed `SparkSQLParser` to be able to handle paths which contains white spaces.

### Why are the changes needed?

This is a bug.
I also have a plan which extends `ADD FILE/JAR/ARCHIVE` to take multiple paths like Hive and the syntax rule change is necessary for that.

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

Yes. Users can pass quoted paths when using `ADD FILE/JAR/ARCHIVE`.

### How was this patch tested?

New test.

Closes #32074 from sarutak/fix-list-files-bug.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-14 10:33:45 +09:00
gengjiaan 27bec91bc9 [SPARK-33604][SQL] Group exception messages in sql/execution
### What changes were proposed in this pull request?
This PR group exception messages in `/core/src/main/scala/org/apache/spark/sql/execution`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31920 from beliefer/SPARK-33604.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 14:03:36 +00:00
Chao Sun 1a6708918b [SPARK-34947][SQL] Streaming write to a V2 table should invalidate its associated cache
### What changes were proposed in this pull request?

Populate table catalog and identifier from `DataStreamWriter` to `WriteToMicroBatchDataSource` so that we can invalidate cache for tables that are updated by a streaming write.

This is somewhat related [SPARK-27484](https://issues.apache.org/jira/browse/SPARK-27484) and [SPARK-34183](https://issues.apache.org/jira/browse/SPARK-34183) (#31700), as ideally we may want to replace `WriteToMicroBatchDataSource` and `WriteToDataSourceV2` with logical write nodes and feed them to analyzer. That will potentially change the code path involved in this PR.

### Why are the changes needed?

Currently `WriteToDataSourceV2` doesn't have cache invalidation logic, and therefore, when the target table for a micro batch streaming job is cached, the cache entry won't be removed when the table is updated.

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

Yes now when a DSv2 table which supports streaming write is updated by a streaming job, its cache will also be invalidated.

### How was this patch tested?

Added a new UT.

Closes #32039 from sunchao/streaming-cache.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 13:31:09 +00:00
Gengliang Wang ade3a1df82 [SPARK-34916][SQL][FOLLOWUP] Remove duplicate code in TreeNode.treePatternBits
### What changes were proposed in this pull request?

Remove duplicate code in `TreeNode.treePatternBits`

### Why are the changes needed?

Code clean up. Make it easier for maintainence.

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

No

### How was this patch tested?

Existing tests.

Closes #32143 from gengliangwang/getBits.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-13 20:25:35 +08:00
Hyukjin Kwon 1f562159bf [SPARK-35045][SQL] Add an internal option to control input buffer in univocity
### What changes were proposed in this pull request?

This PR makes the input buffer configurable (as an internal option). This is mainly to work around uniVocity/univocity-parsers#449.

### Why are the changes needed?

To work around uniVocity/univocity-parsers#449.

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

No, it's only internal option.

### How was this patch tested?

Manually tested by modifying the unittest added in https://github.com/apache/spark/pull/31858 as below:

```diff
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index fd25a79619d..b58f0bd3661 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
 -2460,6 +2460,7  abstract class CSVSuite
       Seq(line).toDF.write.text(path.getAbsolutePath)
       assert(spark.read.format("csv")
         .option("delimiter", "|")
+        .option("inputBufferSize", "128")
         .option("ignoreTrailingWhiteSpace", "true").load(path.getAbsolutePath).count() == 1)
     }
   }
```

Closes #32145 from HyukjinKwon/SPARK-35045.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-13 15:08:01 +03:00
Yingyi Bu 9cd25b46b9 [SPARK-35014] Fix the PhysicalAggregation pattern to not rewrite foldable expressions
### What changes were proposed in this pull request?

Fix PhysicalAggregation to not transform a foldable expression.

### Why are the changes needed?

It can potentially break certain queries like the added unit test shows.

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

Yes, it fixes undesirable errors caused by a returned TypeCheckFailure from places like RegExpReplace.checkInputDataTypes.

Closes #32113 from sigmod/foldable.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 19:57:13 +08:00
Yingyi Bu 49618c9543 [SPARK-35043][SQL] Add condition lambda and rule id to the resolve function family
### What changes were proposed in this pull request?

This PR contains:
- AnalysisHelper changes to allow the resolve function family to stop earlier without traversing the entire tree;
- Example changes in a few rules to support such pruning, e.g., ResolveRandomSeed, ResolveWindowFrame, ResolveWindowOrder, and ResolveNaturalAndUsingJoin.

### Why are the changes needed?

It's a framework-level change for reducing the query compilation time.
In particular, if we update existing analysis rules' call sites as per the examples in this PR, the analysis time can be reduced as described in the [doc](https://docs.google.com/document/d/1SEUhkbo8X-0cYAJFYFDQhxUnKJBz4lLn3u4xR2qfWqk).

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

No.

### How was this patch tested?

It is tested by existing tests.

Closes #32135 from sigmod/resolver.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-13 19:39:11 +08:00
Yuming Wang b34a84e21e [SPARK-34212][SQL][FOLLOWUP] Move the added test to ParquetQuerySuite
### What changes were proposed in this pull request?

This pr moves the added test from `SQLQuerySuite` to `ParquetQuerySuite`.

### Why are the changes needed?
1. It can be tested by `ParquetV1QuerySuite` and `ParquetV2QuerySuite`.
2. Reduce the testing time of `SQLQuerySuite`(SQLQuerySuite ~ 3 min 17 sec, ParquetV1QuerySuite ~ 27 sec).

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

No.

### How was this patch tested?

Unit test.

Closes #32090 from wangyum/SPARK-34212.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 09:04:47 +00:00
Gengliang Wang 5d126537d3 [MINOR][TESTS] Enhance the test instruction of ThriftServerQueryTestSuite
### What changes were proposed in this pull request?

Enhance the test instruction of ThriftServerQueryTestSuite:
1. how to run a single test case
2. how to regenerate golden file for a single test

### Why are the changes needed?

Better documentation.

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

No

### How was this patch tested?

No, just enhance the comments.

Closes #32141 from gengliangwang/updateComment.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-13 16:49:20 +08:00
allisonwang-db 6b8405b574 [SPARK-28379][SQL] Allow non-aggregated single row correlated scalar subquery
### What changes were proposed in this pull request?
This PR allows non-aggregated correlated scalar subquery if the max output row is less than 2. Correlated scalar subqueries need to be aggregated because they are going to be decorrelated and rewritten as LEFT OUTER joins. If the correlated scalar subquery produces more than one output row, the rewrite will yield wrong results.

But this constraint can be relaxed when the subquery plan's the max number of output rows is less than or equal to 1.

### Why are the changes needed?
To relax a constraint in CheckAnalysis for the correlated scalar subquery.

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

### How was this patch tested?
Unit tests

Closes #32111 from allisonwang-db/spark-28379-aggregated.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 07:27:17 +00:00
ayushi agarwal caf33be274 [SPARK-33411][SQL] Cardinality estimation of union, sort and range operator
### What changes were proposed in this pull request?
Supports cardinality estimation of union, sort and range operator.

1. **Union**: number of rows in output will be the sum of number of rows in the output for each child of union, min and max for each column in the output will be the min and max of that particular column coming from its children.
Example:
Table 1
a   b
1   6
2   3
Table 2
a   b
1   3
 4   1
stats for table1 union table2 would be number of rows = 4, columnStats = (a: {min: 1, max: 4}, b: {min: 1, max: 6})

2. **Sort**: row and columns stats would be same as its children.

3. **Range**: number of output rows and distinct count will be equal to number of elements, min and max is calculated from start, end and step param.

### Why are the changes needed?
The change will enhance the feature https://issues.apache.org/jira/browse/SPARK-16026 and will help in other stats based optimizations.

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

### How was this patch tested?
New unit tests added.

Closes #30334 from ayushi-agarwal/SPARK-33411.

Lead-authored-by: ayushi agarwal <ayaga@microsoft.com>
Co-authored-by: ayushi-agarwal <36420535+ayushi-agarwal@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-13 15:17:17 +09:00
Max Gekk 26f312e95f [SPARK-35037][SQL] Recognize sign before the interval string in literals
### What changes were proposed in this pull request?
1. Extend SQL syntax rules to support a sign before the interval strings of ANSI year-month and day-time intervals.
2. Recognize `-` in `AstBuilder` and negate parsed intervals.

### Why are the changes needed?
To conform to the SQL standard which allows a sign before the string interval, see `"5.3 <literal>"`:
```
<interval literal> ::=
  INTERVAL [ <sign> ] <interval string> <interval qualifier>
<interval string> ::=
  <quote> <unquoted interval string> <quote>
<unquoted interval string> ::=
  [ <sign> ] { <year-month literal> | <day-time literal> }
<sign> ::=
    <plus sign>
  | <minus sign>
```

### Does this PR introduce _any_ user-facing change?
Should not because it just extends supported intervals syntax.

### How was this patch tested?
By running new tests in `interval.sql`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32134 from MaxGekk/negative-parsed-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-13 08:55:00 +03:00
Kent Yao 16e2faadac [SPARK-34944][SQL][TESTS] Replace bigint with int for web_returns and store_returns in TPCDS tests to employ correct data type
### What changes were proposed in this pull request?

According to  http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf

```
 2.2.2 Datatype
2.2.2.1 Each column employs one of the following datatypes:
a) Identifier means that the column shall be able to hold any key value generated for that column.
b) Integer means that the column shall be able to exactly represent integer values (i.e., values in increments of
1) in the range of at least ( − 2n − 1) to (2n − 1 − 1), where n is 64.
c) Decimal(d, f) means that the column shall be able to represent decimal values up to and including d digits,
of which f shall occur to the right of the decimal place; the values can be either represented exactly or
interpreted to be in this range.
d) Char(N) means that the column shall be able to hold any string of characters of a fixed length of N.
Comment: If the string that a column of datatype char(N) holds is shorter than N characters, then trailing
spaces shall be stored in the database or the database shall automatically pad with spaces upon retrieval such
that a CHAR_LENGTH() function will return N.
e) Varchar(N) means that the column shall be able to hold any string of characters of a variable length with a
maximum length of N. Columns defined as "varchar(N)" may optionally be implemented as "char(N)".
f) Date means that the column shall be able to express any calendar day between January 1, 1900 and
December 31, 2199.
2.2.2.2 The datatypes do not correspond to any specific SQL-standard datatype. The definitions are provided to
highlight the properties that are required for a particular column. The benchmark implementer may employ any internal representation or SQL datatype that meets those requirements.
```

This PR proposes that we use int for identifiers instead of bigint to reach a compromise with TPC-DS Standard Specification.

After this PR, the field schemas are now consistent with those DDLs in the `tpcds.sql` from tpc-ds tool kit, see https://gist.github.com/yaooqinn/b9978a77bbf4f871a95d6a9103019907

### Why are the changes needed?

reach a compromise with TPC-DS Standard Specification

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

no test only

### How was this patch tested?

test only

Closes #32037 from yaooqinn/SPARK-34944.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-13 11:28:35 +08:00
Gengliang Wang 79e55b44f7 [SPARK-35028][SQL] ANSI mode: disallow group by aliases
### What changes were proposed in this pull request?

Disallow group by aliases under ANSI mode.

### Why are the changes needed?

As per the ANSI SQL standard secion 7.12 <group by clause>:

>Each `grouping column reference` shall unambiguously reference a column of the table resulting from the `from clause`. A column referenced in a `group by clause` is a grouping column.

By forbidding it, we can avoid ambiguous SQL queries like:
```
SELECT col + 1 as col FROM t GROUP BY col
```

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

Yes, group by aliases is not allowed under ANSI mode.

### How was this patch tested?

Unit tests

Closes #32129 from gengliangwang/disallowGroupByAlias.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-13 10:42:57 +08:00
angerszhu 278203d969 [SPARK-28227][SQL] Support projection, aggregate/window functions, and lateral view in the TRANSFORM clause
### What changes were proposed in this pull request?
For Spark SQL, it can't support script transform SQL with aggregationClause/windowClause/LateralView.
This case we can't directly migration Hive SQL to Spark SQL.

In this PR, we treat all script transform statement's query part (exclude transform about part)  as a  separate query block and solve it as ScriptTransformation's child and pass a UnresolvedStart as ScriptTransform's input. Then in analyzer level, we pass child's output as ScriptTransform's input. Then we can support all kind of normal SELECT query combine with script transformation.

Such as transform with aggregation:
```
SELECT TRANSFORM ( d2, max(d1) as max_d1, sum(d3))
USING 'cat' AS (a,b,c)
FROM script_trans
WHERE d1 <= 100
GROUP BY d2
 HAVING max_d1 > 0
```
When we build AST, we treat it as
```
SELECT TRANSFORM (*)
USING 'cat' AS (a,b,c)
FROM (
     SELECT  d2, max(d1) as max_d1, sum(d3)
     FROM script_trans
    WHERE d1 <= 100
    GROUP BY d2
    HAVING max_d1 > 0
) tmp
```
then in Analyzer's `ResolveReferences`, resolve `* (UnresolvedStar)`, then sql behavior like
```
SELECT TRANSFORM ( d2, max(d1) as max_d1, sum(d3))
USING 'cat' AS (a,b,c)
FROM script_trans
WHERE d1 <= 100
GROUP BY d2
HAVING max_d1 > 0
```

About UT, in this pr we add a lot of different SQL to check we can support all kind of such SQL and  each kind of expressions can work well, such as alias, case when, binary compute etc...

### Why are the changes needed?
Support transform with aggregateClause/windowClause/LateralView etc , make sql migration more smoothly

### Does this PR introduce _any_ user-facing change?
User can write transform with  aggregateClause/windowClause/LateralView.

### How was this patch tested?
Added UT

Closes #29087 from AngersZhuuuu/SPARK-28227-NEW.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-13 11:34:45 +09:00
Wenchen Fan 8627cab39d [SPARK-34593][SQL][FOLLOWUP] Fix BroadcastNestedLoopJoinExec.outputPartition with full outer join
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31708 . For full outer join, the final result RDD is created from
```
sparkContext.union(
  matchedStreamRows,
  sparkContext.makeRDD(notMatchedBroadcastRows)
)
```

It's incorrect to say that the final output partitioning is `UnknownPartitioning(left.outputPartitioning.numPartitions)`

### Why are the changes needed?

Fix a correctness bug

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

Yes, see the added test. Fortunately, this bug is not released yet.

### How was this patch tested?

new test

Closes #32132 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-12 17:29:21 -07:00
Yuming Wang e40fce919a [SPARK-34562][SQL] Add test and doc for Parquet Bloom filter push down
### What changes were proposed in this pull request?

This pr add test and document for Parquet Bloom filter push down.

### Why are the changes needed?

Improve document.

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

No.

### How was this patch tested?

Generating docs:
![image](https://user-images.githubusercontent.com/5399861/114327472-c131bb80-9b6b-11eb-87a0-6f9a74eb1097.png)

Closes #32123 from wangyum/SPARK-34562.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-12 17:07:35 +03:00
Max Gekk 8f8bac6435 [SPARK-34905][SQL][TESTS] Enable ANSI intervals in SQLQueryTestSuite/ThriftServerQueryTestSuite
### What changes were proposed in this pull request?
Remove `spark.sql.legacy.interval.enabled` settings from `SQLQueryTestSuite`/`ThriftServerQueryTestSuite` that enables new ANSI intervals by default.

### Why are the changes needed?
To use default settings for intervals, and test new ANSI intervals - year-month and day-time interval introduced by SPARK-27793.

### Does this PR introduce _any_ user-facing change?
Should not because this affects tests only.

### How was this patch tested?
By running the affected tests, for instance:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32099 from MaxGekk/enable-ansi-intervals-sql-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-12 09:25:51 +00:00
Angerszhuuuu 21232377ba [SPARK-33229][SQL] Support partial grouping analytics and concatenated grouping analytics
### What changes were proposed in this pull request?
Support GROUP BY use Separate columns and CUBE/ROLLUP

In postgres sql, it support
```
select a, b, c, count(1) from t group by a, b, cube (a, b, c);
select a, b, c, count(1) from t group by a, b, rollup(a, b, c);
select a, b, c, count(1) from t group by cube(a, b), rollup (a, b, c);
select a, b, c, count(1) from t group by a, b, grouping sets((a, b), (a), ());
```
In this pr, we have done two things as below:

1. Support partial grouping analytics such as `group by a, cube(a, b)`
2. Support mixed grouping analytics such as `group by cube(a, b), rollup(b,c)`

*Partial Groupings*

    Partial Groupings means there are both `group_expression` and `CUBE|ROLLUP|GROUPING SETS`
    in GROUP BY clause. For example:
    `GROUP BY warehouse, CUBE(product, location)` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, product), (warehouse, location), (warehouse))`.
    `GROUP BY warehouse, ROLLUP(product, location)` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, product), (warehouse))`.
    `GROUP BY warehouse, GROUPING SETS((product, location), (producet), ())` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, location), (warehouse))`.

*Concatenated Groupings*

    Concatenated groupings offer a concise way to generate useful combinations of groupings. Groupings specified
    with concatenated groupings yield the cross-product of groupings from each grouping set. The cross-product
    operation enables even a small number of concatenated groupings to generate a large number of final groups.
    The concatenated groupings are specified simply by listing multiple `GROUPING SETS`, `CUBES`, and `ROLLUP`,
    and separating them with commas. For example:
    `GROUP BY GROUPING SETS((warehouse), (producet)), GROUPING SETS((location), (size))` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, location), (warehouse, size), (product, location), (product, size))`.
    `GROUP BY CUBE((warehouse), (producet)), ROLLUP((location), (size))` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product), (warehouse), (producet), ()), GROUPING SETS((location, size), (location), ())`
    `GROUP BY GROUPING SETS(
        (warehouse, product, location, size), (warehouse, product, location), (warehouse, product),
        (warehouse, location, size), (warehouse, location), (warehouse),
        (product, location, size), (product, location), (product),
        (location, size), (location), ())`.
    `GROUP BY order, CUBE((warehouse), (producet)), ROLLUP((location), (size))` is equivalent to
    `GROUP BY order, GROUPING SETS((warehouse, product), (warehouse), (producet), ()), GROUPING SETS((location, size), (location), ())`
    `GROUP BY GROUPING SETS(
        (order, warehouse, product, location, size), (order, warehouse, product, location), (order, warehouse, product),
        (order, warehouse, location, size), (order, warehouse, location), (order, warehouse),
        (order, product, location, size), (order, product, location), (order, product),
        (order, location, size), (order, location), (order))`.

### Why are the changes needed?
Support more flexible grouping analytics

### Does this PR introduce _any_ user-facing change?
User can use sql like
```
select a, b, c, agg_expr() from table group by a, cube(b, c)
```

### How was this patch tested?
Added UT

Closes #30144 from AngersZhuuuu/SPARK-33229.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-12 08:23:52 +00:00
Yingyi Bu 3db8ec258c [SPARK-34916][SQL] Add condition lambda and rule id to the transform family for early stopping
### What changes were proposed in this pull request?

This PR contains:
- TreeNode, QueryPlan, AnalysisHelper changes to allow the transform function family to stop earlier without traversing the entire tree;
- Example changes in a few rules to support such pruning, e.g., ReorderJoin and OptimizeIn.

Here is a [design doc](https://docs.google.com/document/d/1SEUhkbo8X-0cYAJFYFDQhxUnKJBz4lLn3u4xR2qfWqk) that elaborates the ideas and benchmark numbers.

### Why are the changes needed?

It's a framework-level change for reducing the query compilation time.
In particular, if we update existing rules and transform call sites as per the examples in this PR, the analysis time and query optimization time can be reduced as described in this [doc](https://docs.google.com/document/d/1SEUhkbo8X-0cYAJFYFDQhxUnKJBz4lLn3u4xR2qfWqk) .

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

No.

### How was this patch tested?

It is tested by existing tests.

Closes #32060 from sigmod/bits.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-12 11:21:16 +08:00
Max Gekk 90820b3ec3 [SPARK-35017][SQL] Transfer ANSI intervals via Hive Thrift server
### What changes were proposed in this pull request?
1. Map Catalyst's interval types to Hive's types:
    - YearMonthIntervalType -> `interval_year_month`
    - DayTimeIntervalType -> `interval_day_time`
2. Invoke `HiveResult.toHiveString()` to convert external intervals types ` java.time.Period`/`java.time.Duration` to strings.

### Why are the changes needed?
1. To be able to retrieve ANSI intervals via Hive Thrift server.
2. This fixes the issue:
```sql
 $ ./sbin/start-thriftserver.sh
 $ ./bin/beeline
Beeline version 2.3.8 by Apache Hive
beeline> !connect jdbc:hive2://localhost:10000/default "" "" ""
Connecting to jdbc:hive2://localhost:10000/default
Connected to: Spark SQL (version 3.2.0-SNAPSHOT)
0: jdbc:hive2://localhost:10000/default> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
Error: java.lang.IllegalArgumentException: Unrecognized type name: day-time interval (state=,code=0)
```
3. It should unblock https://github.com/apache/spark/pull/32099 which enables `*.sql` tests in `ThriftServerQueryTestSuite`.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
0: jdbc:hive2://localhost:10000/default> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
+----------------------------------------------------+
| subtracttimestamps(TIMESTAMP '2021-01-01 01:02:03.000001', DATE '2020-12-31') |
+----------------------------------------------------+
| 1 01:02:03.000001000                               |
+----------------------------------------------------+
1 row selected (1.637 seconds)
```

### How was this patch tested?
By running new test:
```
$ ./build/sbt -Phive -Phive-thriftserver "test:testOnly *SparkThriftServerProtocolVersionsSuite"
$ ./build/sbt -Phive -Phive-thriftserver "test:testOnly *SparkMetadataOperationSuite"
```
Also checked an array of an interval:
```sql
0: jdbc:hive2://localhost:10000/default> select array(timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31');
+----------------------------------------------------+
| array(subtracttimestamps(TIMESTAMP '2021-01-01 01:02:03.000001', DATE '2020-12-31')) |
+----------------------------------------------------+
| [1 01:02:03.000001000]                             |
+----------------------------------------------------+
```

Closes #32121 from MaxGekk/ansi-intervals-thrift-protocol.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-12 11:56:10 +09:00
Angerszhuuuu 03431d40eb [SPARK-34986][SQL] Make an error msg clearer when ordinal numbers in group-by refer to agg funcs
### What changes were proposed in this pull request?
before when we use aggregate ordinal in group by expression and index position is a aggregate function, it will show error as
```
– !query
select a, b, sum(b) from data group by 3
– !query schema
struct<>
– !query output
org.apache.spark.sql.AnalysisException
aggregate functions are not allowed in GROUP BY, but found sum(data.b)
```

It't not clear enough refactor this error message in this pr

### Why are the changes needed?
refactor  error message

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

### How was this patch tested?
Existed UT

Closes #32089 from AngersZhuuuu/SPARK-34986.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-12 11:45:08 +09:00
Max Gekk 0e761c7307 [SPARK-35016][SQL] Format ANSI intervals in Hive style
### What changes were proposed in this pull request?
1. Extend `IntervalUtils` methods: `toYearMonthIntervalString` and `toDayTimeIntervalString` to support formatting of year-month/day-time intervals in Hive style. The methods get new parameter style which can have to values; `HIVE_STYLE` and `ANSI_STYLE`.
2. Invoke `toYearMonthIntervalString` and `toDayTimeIntervalString` from the `Cast` expression with the `style` parameter is set to `ANSI_STYLE`.
3. Invoke `toYearMonthIntervalString` and `toDayTimeIntervalString` from `HiveResult` with `style` is set to `HIVE_STYLE`.

### Why are the changes needed?
The `spark-sql` shell formats its output in Hive style by using `HiveResult.hiveResultString()`. The changes are needed to match Hive behavior. For instance,

Hive:
```sql
0: jdbc:hive2://localhost:10000/default> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
+-----------------------+
|          _c0          |
+-----------------------+
| 1 01:02:03.000001000  |
+-----------------------+
```

Spark before the changes:
```sql
spark-sql> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
INTERVAL '1 01:02:03.000001' DAY TO SECOND
```

Also this should unblock #32099 which enables *.sql tests in `SQLQueryTestSuite`.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
1 01:02:03.000001000
```

### How was this patch tested?
1. Added new tests to `IntervalUtilsSuite`:
```
$  build/sbt "test:testOnly *IntervalUtilsSuite"
```
2. Modified existing tests in `HiveResultSuite`:
```
$  build/sbt -Phive-2.3 -Phive-thriftserver "testOnly *HiveResultSuite"
```
3. By running cast tests:
```
$ build/sbt "testOnly *CastSuite*"
```

Closes #32120 from MaxGekk/ansi-intervals-hive-thrift-server.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-11 10:13:19 +03:00
Liang-Chi Hsieh 364d1eaf10 [SPARK-34963][SQL] Fix nested column pruning for extracting case-insensitive struct field from array of struct
### What changes were proposed in this pull request?

This patch proposes a fix of nested column pruning for extracting case-insensitive struct field from array of struct.

### Why are the changes needed?

Under case-insensitive mode, nested column pruning rule cannot correctly push down extractor of a struct field of an array of struct, e.g.,

```scala
val query = spark.table("contacts").select("friends.First", "friends.MiDDle")
```

Error stack:
```
[info]   java.lang.IllegalArgumentException: Field "First" does not exist.
[info] Available fields:
[info]   at org.apache.spark.sql.types.StructType$$anonfun$apply$1.apply(StructType.scala:274)
[info]   at org.apache.spark.sql.types.StructType$$anonfun$apply$1.apply(StructType.scala:274)
[info]   at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
[info]   at scala.collection.AbstractMap.getOrElse(Map.scala:59)
[info]   at org.apache.spark.sql.types.StructType.apply(StructType.scala:273)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema$$anonfun$getProjection$3.apply(ProjectionOverSchema.scala:44)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema$$anonfun$getProjection$3.apply(ProjectionOverSchema.scala:41)
```

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

No

### How was this patch tested?

Unit test

Closes #32059 from viirya/fix-array-nested-pruning.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-09 11:52:55 -07:00
Chao Sun ee7bf7d962 [SPARK-35003][SQL] Improve performance for reading smallint in vectorized Parquet reader
### What changes were proposed in this pull request?

Implements `readShorts` in `VectorizedPlainValuesReader`, which decodes `total` shorts in the input buffer at one time, similar to other types.

### Why are the changes needed?

Currently `VectorizedRleValuesReader` reads short integer in the following way:

```java
for (int i = 0; i < n; i++) {
  c.putShort(rowId + i, (short)data.readInteger());
}
```
For PLAIN encoding `data.readInteger` is done via:

```java
public final int readInteger() {
  return getBuffer(4).getInt();
}
```
which means it needs to repeatedly call `slice` buffer for the batch size number of times. This is more expensive than calling it once in a big chunk and then reading the ints out.

Micro benchmark via `DataSourceReadBenchmark` showed ~35% perf improvement.

Before:
```
[info] OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.16
[info] Intel(R) Core(TM) i9-9880H CPU  2.30GHz
[info] SQL Single SMALLINT Column Scan:          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] SQL CSV                                           10249          10271          32          1.5         651.6       1.0X
[info] SQL Json                                           5963           5982          28          2.6         379.1       1.7X
[info] SQL Parquet Vectorized                              141            151          15        111.9           8.9      72.9X
[info] SQL Parquet MR                                     1454           1491          52         10.8          92.4       7.0X
[info] SQL ORC Vectorized                                  160            164           3         98.3          10.2      64.1X
[info] SQL ORC MR                                         1133           1164          44         13.9          72.0       9.0X
```

After:
```
[info] OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.16
[info] Intel(R) Core(TM) i9-9880H CPU  2.30GHz
[info] SQL Single SMALLINT Column Scan:          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] SQL CSV                                           10489          10535          65          1.5         666.8       1.0X
[info] SQL Json                                           5864           5888          34          2.7         372.8       1.8X
[info] SQL Parquet Vectorized                              104            111           8        151.0           6.6     100.7X
[info] SQL Parquet MR                                     1458           1472          20         10.8          92.7       7.2X
[info] SQL ORC Vectorized                                  157            166           7        100.0          10.0      66.7X
[info] SQL ORC MR                                         1121           1147          37         14.0          71.2       9.4X
```

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

No

### How was this patch tested?

Existing tests

Closes #32104 from sunchao/smallint.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-09 08:12:47 -07:00
Ali Afroozeh 0945baf906 [SPARK-34989] Improve the performance of mapChildren and withNewChildren methods
### What changes were proposed in this pull request?
One of the main performance bottlenecks in query compilation is overly-generic tree transformation methods, namely `mapChildren` and `withNewChildren` (defined in `TreeNode`). These methods have an overly-generic implementation to iterate over the children and rely on reflection to create new instances. We have observed that, especially for queries with large query plans, a significant amount of CPU cycles are wasted in these methods. In this PR we make these methods more efficient, by delegating the iteration and instantiation to concrete node types. The benchmarks show that we can expect significant performance improvement in total query compilation time in queries with large query plans (from 30-80%) and about 20% on average.

#### Problem detail
The `mapChildren` method in `TreeNode` is overly generic and costly. To be more specific, this method:
- iterates over all the fields of a node using Scala’s product iterator. While the iteration is not reflection-based, thanks to the Scala compiler generating code for `Product`, we create many anonymous functions and visit many nested structures (recursive calls).
The anonymous functions (presumably compiled to Java anonymous inner classes) also show up quite high on the list in the object allocation profiles, so we are putting unnecessary pressure on GC here.
- does a lot of comparisons. Basically for each element returned from the product iterator, we check if it is a child (contained in the list of children) and then transform it. We can avoid that by just iterating over children, but in the current implementation, we need to gather all the fields (only transform the children) so that we can instantiate the object using the reflection.
- creates objects using reflection, by delegating to the `makeCopy` method, which is several orders of magnitude slower than using the constructor.

#### Solution
The proposed solution in this PR is rather straightforward: we rewrite the `mapChildren` method using the `children` and `withNewChildren` methods. The default `withNewChildren` method suffers from the same problems as `mapChildren` and we need to make it more efficient by specializing it in concrete classes.  Similar to how each concrete query plan node already defines its children, it should also define how they can be constructed given a new list of children. Actually, the implementation is quite simple in most cases and is a one-liner thanks to the copy method present in Scala case classes. Note that we cannot abstract over the copy method, it’s generated by the compiler for case classes if no other type higher in the hierarchy defines it. For most concrete nodes, the implementation of `withNewChildren` looks like this:
```
override def withNewChildren(newChildren: Seq[LogicalPlan]): LogicalPlan = copy(children = newChildren)
```
The current `withNewChildren` method has two properties that we should preserve:

- It returns the same instance if the provided children are the same as its children, i.e., it preserves referential equality.
- It copies tags and maintains the origin links when a new copy is created.

These properties are hard to enforce in the concrete node type implementation. Therefore, we propose a template method `withNewChildrenInternal` that should be rewritten by the concrete classes and let the `withNewChildren` method take care of referential equality and copying:
```
override def withNewChildren(newChildren: Seq[LogicalPlan]): LogicalPlan = {
 if (childrenFastEquals(children, newChildren)) {
   this
 } else {
   CurrentOrigin.withOrigin(origin) {
     val res = withNewChildrenInternal(newChildren)
     res.copyTagsFrom(this)
     res
   }
 }
}
```

With the refactoring done in a previous PR (https://github.com/apache/spark/pull/31932) most tree node types fall in one of the categories of `Leaf`, `Unary`, `Binary` or `Ternary`. These traits have a more efficient implementation for `mapChildren` and define a more specialized version of `withNewChildrenInternal` that avoids creating unnecessary lists. For example, the `mapChildren` method in `UnaryLike` is defined as follows:
```
  override final def mapChildren(f: T => T): T = {
    val newChild = f(child)
    if (newChild fastEquals child) {
      this.asInstanceOf[T]
    } else {
      CurrentOrigin.withOrigin(origin) {
        val res = withNewChildInternal(newChild)
        res.copyTagsFrom(this.asInstanceOf[T])
        res
      }
    }
  }
```

#### Results
With this PR, we have observed significant performance improvements in query compilation time, more specifically in the analysis and optimization phases. The table below shows the TPC-DS queries that had more than 25% speedup in compilation times. Biggest speedups are observed in queries with large query plans.
| Query  | Speedup |
| ------------- | ------------- |
|q4    |29%|
|q9    |81%|
|q14a  |31%|
|q14b  |28%|
|q22   |33%|
|q33   |29%|
|q34   |25%|
|q39   |27%|
|q41   |27%|
|q44   |26%|
|q47   |28%|
|q48   |76%|
|q49   |46%|
|q56   |26%|
|q58   |43%|
|q59   |46%|
|q60   |50%|
|q65   |59%|
|q66   |46%|
|q67   |52%|
|q69   |31%|
|q70   |30%|
|q96   |26%|
|q98   |32%|

#### Binary incompatibility
Changing the `withNewChildren` in `TreeNode` breaks the binary compatibility of the code compiled against older versions of Spark because now it is expected that concrete `TreeNode` subclasses all implement the `withNewChildrenInternal` method. This is a problem, for example, when users write custom expressions. This change is the right choice, since it forces all newly added expressions to Catalyst implement it in an efficient manner and will prevent future regressions.
Please note that we have not completely removed the old implementation and renamed it to `legacyWithNewChildren`. This method will be removed in the future and for now helps the transition. There are expressions such as `UpdateFields` that have a complex way of defining children. Writing `withNewChildren` for them requires refactoring the expression. For now, these expressions use the old, slow method. In a future PR we address these expressions.

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

This PR does not introduce user facing changes but my break binary compatibility of the code compiled against older versions. See the binary compatibility section.

### How was this patch tested?

This PR is mainly a refactoring and passes existing tests.

Closes #32030 from dbaliafroozeh/ImprovedMapChildren.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-09 15:06:26 +02:00
Gengliang Wang bfba7fadd2 [SPARK-34881][SQL][FOLLOWUP] Implement toString() and sql() methods for TRY_CAST
### What changes were proposed in this pull request?

Implement toString() and sql() methods for TRY_CAST

### Why are the changes needed?

The new expression should have a different name from `CAST` in SQL/String representation.

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

Yes, in the result of `explain()`, users can see try_cast if the new expression is used.

### How was this patch tested?

Unit tests.

Closes #32098 from gengliangwang/tryCastString.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-09 15:39:25 +08:00
Chao Sun 5013171fd3 [SPARK-34973][SQL] Cleanup unused fields and methods in vectorized Parquet reader
### What changes were proposed in this pull request?

Remove some unused fields and methods in `SpecificParquetRecordReaderBase` and `VectorizedColumnReader`.

### Why are the changes needed?

Some fields and methods in these classes are no longer used since years ago. It's better to clean them up to make the code easier to maintain and read.

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

No

### How was this patch tested?

Existing tests

Closes #32071 from sunchao/cleanup-parquet.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-08 11:21:07 -07:00
Max Gekk 96a3533de8 [SPARK-34984][SQL] ANSI intervals formatting in hive results
### What changes were proposed in this pull request?
Extend `HiveResult.toHiveString()` to support new interval types `YearMonthIntervalType` and `DayTimeIntervalType`.

### Why are the changes needed?
To fix failures while formatting ANSI intervals as Hive strings. For example:
```sql
spark-sql> select timestamp'now' - date'2021-01-01';
21/04/08 09:42:49 ERROR SparkSQLDriver: Failed in [select timestamp'now' - date'2021-01-01']
scala.MatchError: (PT2337H42M46.649S,DayTimeIntervalType) (of class scala.Tuple2)
	at org.apache.spark.sql.execution.HiveResult$.toHiveString(HiveResult.scala:97)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> select timestamp'now' - date'2021-01-01';
INTERVAL '97 09:37:52.171' DAY TO SECOND
```

### How was this patch tested?
By running new tests:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "testOnly *HiveResultSuite"
```

Closes #32087 from MaxGekk/ansi-interval-hiveResultString.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 16:18:15 +00:00
Tathagata Das c1c9a318c2 [SPARK-34962][SQL] Explicit representation of * in UpdateAction and InsertAction in MergeIntoTable
### What changes were proposed in this pull request?
Change UpdateAction and InsertAction of MergeIntoTable to explicitly represent star,

### Why are the changes needed?
Currently, UpdateAction and InsertAction in the MergeIntoTable implicitly represent `update set *` and `insert *` with empty assignments. That means there is no way to differentiate between the representations of "update all columns" and "update no columns". For SQL MERGE queries, this inability does not matter because the SQL MERGE grammar that generated the MergeIntoTable plan does not allow "update no columns". However, other ways of generating the MergeIntoTable plan may not have that limitation, and may want to allow specifying "update no columns". For example, in the Delta Lake project we provide a type-safe Scala API for Merge, where it is perfectly valid to produce a Merge query with an update clause but no update assignments. Currently, we cannot use MergeIntoTable to represent this plan, thus complicating the generation, and resolution of merge query from scala API.

Side note: fixed another bug where a merge plan with star and no other expressions with unresolved attributes (e.g. all non-optional predicates are `literal(true)`), then resolution will be skipped and star wont expanded. added test for that.

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

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

Closes #32067 from tdas/SPARK-34962-2.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 14:21:31 +00:00
Angerszhuuuu 90613df652 [SPARK-33233][SQL] CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal
### What changes were proposed in this pull request?
Currently, we can't support use ordinal in CUBE/ROLLUP/GROUPING SETS,
this pr make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

### Why are the changes needed?
Make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal.
Postgres SQL and TeraData support this use case.

### Does this PR introduce _any_ user-facing change?
User can use ordinal in CUBE/ROLLUP/GROUPING SETS, such as
```
-- can use ordinal in CUBE
select a, b, count(1) from data group by cube(1, 2);

-- mixed cases: can use ordinal in CUBE
select a, b, count(1) from data group by cube(1, b);

-- can use ordinal with cube
select a, b, count(1) from data group by 1, 2 with cube;

-- can use ordinal in ROLLUP
select a, b, count(1) from data group by rollup(1, 2);

-- mixed cases: can use ordinal in ROLLUP
select a, b, count(1) from data group by rollup(1, b);

-- can use ordinal with rollup
select a, b, count(1) from data group by 1, 2 with rollup;

-- can use ordinal in GROUPING SETS
select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));

-- mixed cases: can use ordinal in GROUPING SETS
select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));

select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));

```

### How was this patch tested?
Added UT

Closes #30145 from AngersZhuuuu/SPARK-33233.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 14:19:41 +00:00
allisonwang-db ac01070a77 [SPARK-34946][SQL] Block unsupported correlated scalar subquery in Aggregate
### What changes were proposed in this pull request?
This PR adds two additional checks in `CheckAnalysis` for correlated scalar subquery in Aggregate. It blocks the cases that Spark do not currently support based on the rewrite logic in `RewriteCorrelatedScalarSubquery`:
aff6c0febb/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala (L618-L624)

### Why are the changes needed?
It can be confusing to users when their queries pass the check analysis but cannot be executed. Also, the error messages are confusing:

#### Case 1: correlated scalar subquery in the grouping expressions but not in aggregate expressions

```sql
SELECT SUM(c2) FROM t t1 GROUP BY (SELECT SUM(c2) FROM t t2 WHERE t1.c1 = t2.c1)
```
We get this error:
```
java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; something went wrong in analysis
```
because the correlated scalar subquery is not rewritten properly:
```scala
== Optimized Logical Plan ==
Aggregate [scalar-subquery#5 [(c1#6 = c1#6#93)]], [sum(c2#7) AS sum(c2)#11L]
:  +- Aggregate [c1#6], [sum(c2#7) AS sum(c2)#15L, c1#6 AS c1#6#93]
:     +- LocalRelation [c1#6, c2#7]
+- LocalRelation [c1#6, c2#7]
```

#### Case 2: correlated scalar subquery in the aggregate expressions but not in the grouping expressions

```sql
SELECT (SELECT SUM(c2) FROM t t2 WHERE t1.c1 = t2.c1), SUM(c2) FROM t t1 GROUP BY c1
```
We get this error:
```
java.lang.IllegalStateException: Couldn't find sum(c2)#69L in [c1#60,sum(c2#61)#64L]
```
because the transformed correlated scalar subquery output is not present in the grouping expression of the Aggregate:
```scala
== Optimized Logical Plan ==
Aggregate [c1#60], [sum(c2)#69L AS scalarsubquery(c1)#70L, sum(c2#61) AS sum(c2)#65L]
+- Project [c1#60, c2#61, sum(c2)#69L]
   +- Join LeftOuter, (c1#60 = c1#60#95)
      :- LocalRelation [c1#60, c2#61]
      +- Aggregate [c1#60], [sum(c2#61) AS sum(c2)#69L, c1#60 AS c1#60#95]
         +- LocalRelation [c1#60, c2#61]
```

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

### How was this patch tested?
New unit tests

Closes #32054 from allisonwang-db/spark-34946-scalar-subquery-agg.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 13:03:08 +00:00
Kousuke Saruta e5d972e84e [SPARK-34955][SQL] ADD JAR command cannot add jar files which contains whitespaces in the path
### What changes were proposed in this pull request?

This PR fixes an issue that `ADD JAR` command can't add jar files which contain whitespaces in the path though `ADD FILE` and `ADD ARCHIVE` work with such files.

If we have `/some/path/test file.jar` and execute the following command:

```
ADD JAR "/some/path/test file.jar";
```

The following exception is thrown.

```
21/04/05 10:40:38 ERROR SparkSQLDriver: Failed in [add jar "/some/path/test file.jar"]
java.lang.IllegalArgumentException: Illegal character in path at index 9: /some/path/test file.jar
	at java.net.URI.create(URI.java:852)
	at org.apache.spark.sql.hive.HiveSessionResourceLoader.addJar(HiveSessionStateBuilder.scala:129)
	at org.apache.spark.sql.execution.command.AddJarCommand.run(resources.scala:34)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
```

This is because `HiveSessionStateBuilder` and `SessionStateBuilder` don't check whether the form of the path is URI or plain path and it always regards the path as URI form.
Whitespces should be encoded to `%20` so `/some/path/test file.jar` is rejected.
We can resolve this part by checking whether the given path is URI form or not.

Unfortunatelly, if we fix this part, another problem occurs.
When we execute `ADD JAR` command, Hive's `ADD JAR` command is executed in `HiveClientImpl.addJar` and `AddResourceProcessor.run` is transitively invoked.
In `AddResourceProcessor.run`, the command line is just split by `
s+` and the path is also split into `/some/path/test` and `file.jar` and passed to `ss.add_resources`.
f1e8713703/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java (L56-L75)
So, the command still fails.

Even if we convert the form of the path to URI like `file:/some/path/test%20file.jar` and execute the following command:

```
ADD JAR "file:/some/path/test%20file";
```

The following exception is thrown.

```
21/04/05 10:40:53 ERROR SessionState: file:/some/path/test%20file.jar does not exist
java.lang.IllegalArgumentException: file:/some/path/test%20file.jar does not exist
	at org.apache.hadoop.hive.ql.session.SessionState.validateFiles(SessionState.java:1168)
	at org.apache.hadoop.hive.ql.session.SessionState$ResourceType.preHook(SessionState.java:1289)
	at org.apache.hadoop.hive.ql.session.SessionState$ResourceType$1.preHook(SessionState.java:1278)
	at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1378)
	at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1336)
	at org.apache.hadoop.hive.ql.processors.AddResourceProcessor.run(AddResourceProcessor.java:74)
```

The reason is `Utilities.realFile` invoked in `SessionState.validateFiles` returns `null` as the result of `fs.exists(path)` is `false`.
f1e8713703/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (L1052-L1064)

`fs.exists` checks the existence of the given path by comparing the string representation of Hadoop's `Path`.
The string representation of `Path` is similar to URI but it's actually different.
`Path` doesn't encode the given path.
For example, the URI form of `/some/path/jar file.jar` is `file:/some/path/jar%20file.jar` but the `Path` form of it is `file:/some/path/jar file.jar`. So `fs.exists` returns false.

So the solution I come up with is removing Hive's `ADD JAR` from `HiveClientimpl.addJar`.
I think Hive's `ADD JAR` was used to add jar files to the class loader for metadata and isolate the class loader from the one for execution.
https://github.com/apache/spark/pull/6758/files#diff-cdb07de713c84779a5308f65be47964af865e15f00eb9897ccf8a74908d581bbR94-R103

But, as of SPARK-10810 and SPARK-10902 (#8909) are resolved, the class loaders for metadata and execution seem to be isolated with different way.
https://github.com/apache/spark/pull/8909/files#diff-8ef7cabf145d3fe7081da799fa415189d9708892ed76d4d13dd20fa27021d149R635-R641

In the current implementation, such class loaders seem to be isolated by `SharedState.jarClassLoader` and `IsolatedClientLoader.classLoader`.

https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala#L173-L188
https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L956-L967

So I wonder we can remove Hive's `ADD JAR` from `HiveClientImpl.addJar`.
### Why are the changes needed?

This is a bug.

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

### How was this patch tested?

Closes #32052 from sarutak/add-jar-whitespace.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-07 11:43:03 -07:00
Max Gekk 3dfd456b2c [SPARK-34668][SQL] Support casting of day-time intervals to strings
### What changes were proposed in this pull request?
1. Added new method `toDayTimeIntervalString()` to `IntervalUtils` which converts a day-time interval as a number of microseconds to a string in the form **"INTERVAL '[sign]days hours:minutes:secondsWithFraction' DAY TO SECOND"**.
2. Extended the `Cast` expression to support casting of `DayTimeIntervalType` to `StringType`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such casting.

### Does this PR introduce _any_ user-facing change?
Should not because new day-time interval has not been released yet.

### How was this patch tested?
Added new tests for casting:
```
$ build/sbt "testOnly *CastSuite*"
```

Closes #32070 from MaxGekk/cast-dt-interval-to-string.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-07 13:28:55 +00:00
Angerszhuuuu 5a3f41a017 [SPARK-34976][SQL] Rename GroupingSet to BaseGroupingSets
### What changes were proposed in this pull request?
Current trait `GroupingSet` is ambiguous, since `grouping set` in parser level means one set of a group.
Rename this to `BaseGroupingSets` since cube/rollup is syntax sugar for grouping sets.`

### Why are the changes needed?
Refactor class name

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

### How was this patch tested?
Not need

Closes #32073 from AngersZhuuuu/SPARK-34976.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-07 13:27:21 +00:00
Gengliang Wang f208d80881 [SPARK-34970][SQL][SERCURITY] Redact map-type options in the output of explain()
### What changes were proposed in this pull request?

The `explain()` method prints the arguments of tree nodes in logical/physical plans. The arguments could contain a map-type option that contains sensitive data.
We should map-type options in the output of `explain()`. Otherwise, we will see sensitive data in explain output or Spark UI.
![image](https://user-images.githubusercontent.com/1097932/113719178-326ffb00-96a2-11eb-8a2c-28fca3e72941.png)

### Why are the changes needed?

Data security.

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

Yes, redact the map-type options in the output of `explain()`

### How was this patch tested?

Unit tests

Closes #32066 from gengliangwang/redactOptions.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-07 18:19:01 +08:00
Ryan Blue 3c7d6c38e8 [SPARK-27658][SQL] Add FunctionCatalog API
## What changes were proposed in this pull request?

This adds a new API for catalog plugins that exposes functions to Spark. The API can list and load functions. This does not include create, delete, or alter operations.
- [Design Document](https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit?usp=sharing)

There are 3 types of functions defined:
* A `ScalarFunction` that produces a value for every call
* An `AggregateFunction` that produces a value after updates for a group of rows

Functions loaded from the catalog by name as `UnboundFunction`. Once input arguments are determined `bind` is called on the unbound function to get a `BoundFunction` implementation that is one of the 3 types above. Binding can fail if the function doesn't support the input type. `BoundFunction` returns the result type produced by the function.

## How was this patch tested?

This includes a test that demonstrates the new API.

Closes #24559 from rdblue/SPARK-27658-add-function-catalog-api.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-07 09:19:20 +00:00
Ali Afroozeh 06c09a79b3 [SPARK-34969][SPARK-34906][SQL] Followup for Refactor TreeNode's children handling methods into specialized traits
### What changes were proposed in this pull request?

This is a followup for https://github.com/apache/spark/pull/31932.
In this PR we:
- Introduce the `QuaternaryLike` trait for node types with 4 children.
- Specialize more node types
- Fix a number of style errors that were introduced in the original PR.

### Why are the changes needed?

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

### How was this patch tested?

This is a refactoring, passes existing tests.

Closes #32065 from dbaliafroozeh/FollowupSPARK-34906.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-07 09:50:30 +02:00
allisonwang-db 0aa2c284e4 [SPARK-34678][SQL] Add table function registry
### What changes were proposed in this pull request?
This PR extends the current function registry and catalog to support table-valued functions by adding a table function registry. It also refactors `range` to be a built-in function in the table function registry.

### Why are the changes needed?
Currently, Spark resolves table-valued functions very differently from the other functions. This change is to make the behavior for table and non-table functions consistent. It also allows Spark to display information about built-in table-valued functions:
Before:
```scala
scala> sql("describe function range").show(false)
+--------------------------+
|function_desc             |
+--------------------------+
|Function: range not found.|
+--------------------------+
```
After:
```scala
Function: range
Class: org.apache.spark.sql.catalyst.plans.logical.Range
Usage:
  range(start: Long, end: Long, step: Long, numPartitions: Int)
  range(start: Long, end: Long, step: Long)
  range(start: Long, end: Long)
  range(end: Long)

// Extended
Function: range
Class: org.apache.spark.sql.catalyst.plans.logical.Range
Usage:
  range(start: Long, end: Long, step: Long, numPartitions: Int)
  range(start: Long, end: Long, step: Long)
  range(start: Long, end: Long)
  range(end: Long)

Extended Usage:
  Examples:
    > SELECT * FROM range(1);
      +---+
      | id|
      +---+
      |  0|
      +---+
    > SELECT * FROM range(0, 2);
      +---+
      |id |
      +---+
      |0  |
      |1  |
      +---+
    > SELECT range(0, 4, 2);
      +---+
      |id |
      +---+
      |0  |
      |2  |
      +---+

    Since: 2.0.0
```

### Does this PR introduce _any_ user-facing change?
Yes. User will not be able to create a function with name `range` in the default database:
Before:
```scala
scala> sql("create function range as 'range'")
res3: org.apache.spark.sql.DataFrame = []
```
After:
```
scala> sql("create function range as 'range'")
org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException: Function 'default.range' already exists in database 'default'
```

### How was this patch tested?
Unit test

Closes #31791 from allisonwang-db/spark-34678-table-func-registry.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-07 05:49:36 +00:00
Tanel Kiis 7c8dc5e0b5 [SPARK-34922][SQL] Use a relative cost comparison function in the CBO
### What changes were proposed in this pull request?

Changed the cost comparison function of the CBO to use the ratios of row counts and sizes in bytes.

### Why are the changes needed?

In #30965 we changed to CBO cost comparison function so it would be "symetric": `A.betterThan(B)` now implies, that `!B.betterThan(A)`.
With that we caused a performance regressions in some queries - TPCDS q19 for example.

The original cost comparison function used the ratios `relativeRows = A.rowCount / B.rowCount` and `relativeSize = A.size / B.size`. The changed function compared "absolute" cost values `costA = w*A.rowCount + (1-w)*A.size` and `costB = w*B.rowCount + (1-w)*B.size`.

Given the input from wzhfy we decided to go back to the relative values, because otherwise one (size) may overwhelm the other (rowCount). But this time we avoid adding up the ratios.

Originally `A.betterThan(B) => w*relativeRows + (1-w)*relativeSize < 1` was used. Besides being "non-symteric", this also can exhibit one overwhelming other.
For `w=0.5` If `A` size (bytes) is at least 2x larger than `B`, then no matter how many times more rows does the `B` plan have, `B` will allways be considered to be better - `0.5*2 + 0.5*0.00000000000001 > 1`.

When working with ratios, then it would be better to multiply them.
The proposed cost comparison function is: `A.betterThan(B) => relativeRows^w  * relativeSize^(1-w) < 1`.

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

Comparison of the changed TPCDS v1.4 query execution times at sf=10:

  | absolute | multiplicative |   | additive |  
-- | -- | -- | -- | -- | --
q12 | 145 | 137 | -5.52% | 141 | -2.76%
q13 | 264 | 271 | 2.65% | 271 | 2.65%
q17 | 4521 | 4243 | -6.15% | 4348 | -3.83%
q18 | 758 | 466 | -38.52% | 480 | -36.68%
q19 | 38503 | 2167 | -94.37% | 2176 | -94.35%
q20 | 119 | 120 | 0.84% | 126 | 5.88%
q24a | 16429 | 16838 | 2.49% | 17103 | 4.10%
q24b | 16592 | 16999 | 2.45% | 17268 | 4.07%
q25 | 3558 | 3556 | -0.06% | 3675 | 3.29%
q33 | 362 | 361 | -0.28% | 380 | 4.97%
q52 | 1020 | 1032 | 1.18% | 1052 | 3.14%
q55 | 927 | 938 | 1.19% | 961 | 3.67%
q72 | 24169 | 13377 | -44.65% | 24306 | 0.57%
q81 | 1285 | 1185 | -7.78% | 1168 | -9.11%
q91 | 324 | 336 | 3.70% | 337 | 4.01%
q98 | 126 | 129 | 2.38% | 131 | 3.97%

All times are in ms, the change is compared to the situation in the master branch (absolute).
The proposed cost function (multiplicative) significantlly improves the performance on q18, q19 and q72. The original cost function (additive) has similar improvements at q18 and q19. All other chagnes are within the error bars and I would ignore them - perhaps q81 has also improved.

### How was this patch tested?

PlanStabilitySuite

Closes #32014 from tanelk/SPARK-34922_cbo_better_cost_function.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-07 11:31:10 +09:00
Max Gekk 4b5fc1da75 [SPARK-34667][SQL] Support casting of year-month intervals to strings
### What changes were proposed in this pull request?
1. Added new method `toYearMonthIntervalString()` to `IntervalUtils` which converts an year-month interval as a number of month to a string in the form **"INTERVAL '[sign]yearField-monthField' YEAR TO MONTH"**.
2. Extended the `Cast` expression to support casting of `YearMonthIntervalType` to `StringType`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such casting.

### Does this PR introduce _any_ user-facing change?
Should not because new year-month interval has not been released yet.

### How was this patch tested?
Added new tests for casting:
```
$ build/sbt "testOnly *CastSuite*"
```

Closes #32056 from MaxGekk/cast-ym-interval-to-string.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-06 17:59:50 +03:00
Wenchen Fan 19c7d2f3d8 Revert "[SPARK-34884][SQL] Improve DPP evaluation to make filtering side must can broadcast by size or broadcast by hint"
This reverts commit de66fa63f9.
2021-04-06 22:58:41 +08:00
Karen Feng 3b634f66c3 [SPARK-34923][SQL] Metadata output should be empty for more plans
### What changes were proposed in this pull request?

Changes the metadata propagation framework.

Previously, most `LogicalPlan`'s propagated their `children`'s `metadataOutput`. This did not make sense in cases where the `LogicalPlan` did not even propagate their `children`'s `output`.

I set the metadata output for plans that do not propagate their `children`'s `output` to be `Nil`. Notably, `Project` and `View` no longer have metadata output.

### Why are the changes needed?

Previously, `SELECT m from (SELECT a from tb)` would output `m` if it were metadata. This did not make sense.

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

Yes. Now, `SELECT m from (SELECT a from tb)` will encounter an `AnalysisException`.

### How was this patch tested?

Added unit tests. I did not cover all cases, as they are fairly extensive. However, the new tests cover major cases (and an existing test already covers Join).

Closes #32017 from karenfeng/spark-34923.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-06 16:04:30 +08:00
Kent Yao 7cffacef18 [SPARK-34935][SQL] CREATE TABLE LIKE should respect the reserved table properties
### What changes were proposed in this pull request?

CREATE TABLE LIKE should respect the reserved properties of tables and fail if specified, using `spark.sql.legacy.notReserveProperties` to restore.

### Why are the changes needed?

Make DDLs consistently treat reserved properties

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

YES, this is a breaking change as using `create table like` w/ reserved properties will fail.

### How was this patch tested?

new test

Closes #32025 from yaooqinn/SPARK-34935.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-06 08:52:48 +09:00
Wenchen Fan 39d5677ee3 [SPARK-34932][SQL] deprecate GROUP BY ... GROUPING SETS (...) and promote GROUP BY GROUPING SETS (...)
### What changes were proposed in this pull request?

GROUP BY ... GROUPING SETS (...) is a weird SQL syntax we copied from Hive. It's not in the SQL standard or any other mainstream databases. This syntax requires users to repeat the expressions inside `GROUPING SETS (...)` after `GROUP BY`, and has a weird null semantic if `GROUP BY` contains extra expressions than `GROUPING SETS (...)`.

This PR deprecates this syntax:
1. Do not promote it in the document and only mention it as a Hive compatible sytax.
2. Simplify the code to only keep it for Hive compatibility.

### Why are the changes needed?

Deprecate a weird grammar.

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

No breaking change, but it removes a check to simplify the code: `GROUP BY a GROUPING SETS(a, b)` fails before and forces users to also put `b` after `GROUP BY`. Now this works just as `GROUP BY GROUPING SETS(a, b)`.

### How was this patch tested?

existing tests

Closes #32022 from cloud-fan/followup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-06 08:49:08 +09:00
Dongjoon Hyun 748f05fca9 [SPARK-34954][SQL] Use zstd codec name in ORC file names
### What changes were proposed in this pull request?

This PR aims to add `zstd` codec names in the Spark generated ORC file names for consistency.

### Why are the changes needed?

Like the other ORC supported codecs, we had better have `zstd` in the Spark generated ORC file names. Please note that there is no problem at reading/writing ORC zstd files currently. This PR only aims to revise the file name format for consistency.

**SNAPPY**
```
scala> spark.range(10).repartition(1).write.option("compression", "snappy").orc("/tmp/snappy")

$ ls -al /tmp/snappy
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:17 .
drwxrwxrwt  14 root      wheel  448 Apr  4 12:17 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:17 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:17 .part-00000-833bb7ad-d1e1-48cc-9719-07b2d594aa4c-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:17 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:17 part-00000-833bb7ad-d1e1-48cc-9719-07b2d594aa4c-c000.snappy.orc
```

**ZSTD (AS-IS)**
```
scala> spark.range(10).repartition(1).write.option("compression", "zstd").orc("/tmp/zstd")

$ ls -al /tmp/zstd
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:17 .
drwxrwxrwt  14 root      wheel  448 Apr  4 12:17 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:17 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:17 .part-00000-2f403ce9-7314-4db5-bca3-b1c1dd83335f-c000.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:17 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:17 part-00000-2f403ce9-7314-4db5-bca3-b1c1dd83335f-c000.orc
```

**ZSTD (After this PR)**
```
scala> spark.range(10).repartition(1).write.option("compression", "zstd").orc("/tmp/zstd_new")

$ ls -al /tmp/zstd_new
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:28 .
drwxrwxrwt  15 root      wheel  480 Apr  4 12:28 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:28 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:28 .part-00000-49d57329-7196-4caf-839c-4251c876e26b-c000.zstd.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:28 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:28 part-00000-49d57329-7196-4caf-839c-4251c876e26b-c000.zstd.orc
```

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

No.

### How was this patch tested?

Pass the CIs with the updated UT.

Closes #32051 from dongjoon-hyun/SPARK-34954.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-04 17:11:56 -07:00
HyukjinKwon ebf01ec3c1 [SPARK-34950][TESTS] Update benchmark results to the ones created by GitHub Actions machines
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/32015 added a way to run benchmarks much more easily in the same GitHub Actions build. This PR updates the benchmark results by using the way.

**NOTE** that looks like GitHub Actions use four types of CPU given my observations:

- Intel(R) Xeon(R) Platinum 8171M CPU  2.60GHz
- Intel(R) Xeon(R) CPU E5-2673 v4  2.30GHz
- Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
- Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz

Given my quick research, seems like they perform roughly similarly:

![Screen Shot 2021-04-03 at 9 31 23 PM](https://user-images.githubusercontent.com/6477701/113478478-f4b57b80-94c3-11eb-9047-f81ca8c59672.png)

I couldn't find enough information about Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz but the performance seems roughly similar given the numbers.

So shouldn't be a big deal especially given that this way is much easier, encourages contributors to run more and guarantee the same number of cores and same memory with the same softwares.

### Why are the changes needed?

To have a base line of the benchmarks accordingly.

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

No, dev-only.

### How was this patch tested?

It was generated from:

- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

Closes #32044 from HyukjinKwon/SPARK-34950.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-03 23:02:56 +03:00
HyukjinKwon 71effba5f2 [SPARK-34821][INFRA] Set up a workflow for developers to run benchmark in their fork
### What changes were proposed in this pull request?

This PR proposes to add a workflow that allows developers to run benchmarks and download the results files.  After this PR, developers can run benchmarks in GitHub Actions in their fork.

### Why are the changes needed?

1. Very easy to use.
2. We can use the (almost) same environment to run the benchmarks. Given my few experiments and observation, the CPU, cores, and memory are same.
3. Does not burden ASF's resource at GitHub Actions.

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

No, dev-only.

### How was this patch tested?

Manually tested in https://github.com/HyukjinKwon/spark/pull/31.

Entire benchmarks are being run as below:
- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

### How do developers use it in their fork?

1. **Go to Actions in your fork, and click "Run benchmarks"**

    ![Screen Shot 2021-03-31 at 10 15 13 PM](https://user-images.githubusercontent.com/6477701/113150018-99d71680-926e-11eb-8647-4ecf062c55f2.png)

2. **Run the benchmarks with JDK 8 or 11 with benchmark classes to run. Glob pattern is supported just like `testOnly` in SBT**

    ![Screen Shot 2021-04-02 at 8 35 02 PM](https://user-images.githubusercontent.com/6477701/113412599-ab95f680-93f3-11eb-9a15-c6ed54587b9d.png)

3. **After finishing the jobs, the benchmark results are available on the top in the underlying workflow:**

    ![Screen Shot 2021-03-31 at 10 17 21 PM](https://user-images.githubusercontent.com/6477701/113150332-ede1fb00-926e-11eb-9c0e-97d195070508.png)

4. **After downloading it, unzip and untar at Spark git root directory:**

    ```bash
    cd .../spark
    mv ~/Downloads/benchmark-results-8.zip .
    unzip benchmark-results-8.zip
    tar -xvf benchmark-results-8.tar
    ```

5. **Check the results:**

    ```bash
    git status
    ```

    ```
    ...
        modified:   core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
    ```

Closes #32015 from HyukjinKwon/SPARK-34821-pr.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-03 20:55:54 +09:00
Chao Sun f1d42bb68d [SPARK-34945][DOC] Fix Javadoc for classes in catalyst module
### What changes were proposed in this pull request?

Use proper Java doc format for Java classes within `catalyst` module

### Why are the changes needed?

Many Java classes in `catalyst`, especially those for DataSource V2, do not have proper Java doc format. By fixing the format it helps to improve the doc's readability.

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

No

### How was this patch tested?

N/A

Closes #32038 from sunchao/javadoc.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-02 23:00:19 -07:00
Angerszhuuuu 65da9287bc [SPARK-34926][SQL] PartitioningUtils.getPathFragment() should respect partition value is null
### What changes were proposed in this pull request?

When we insert data into a partition table partition with empty DataFrame. We will call `PartitioningUtils.getPathFragment()`
then to update this partition's metadata too.
When we insert to a partition when partition value is `null`, it will throw exception like
```
[info]   java.lang.NullPointerException:
[info]   at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:51)
[info]   at scala.collection.immutable.StringOps.length(StringOps.scala:51)
[info]   at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:35)
[info]   at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
[info]   at scala.collection.immutable.StringOps.foreach(StringOps.scala:33)
[info]   at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.escapePathName(ExternalCatalogUtils.scala:69)
[info]   at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.getPartitionValueString(ExternalCatalogUtils.scala:126)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.$anonfun$getPathFragment$1(PartitioningUtils.scala:354)
[info]   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[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 scala.collection.IterableLike.foreach(IterableLike.scala:74)
[info]   at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
```
`PartitioningUtils.getPathFragment()`  should support `null` value too

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32018 from AngersZhuuuu/SPARK-34926.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-02 10:26:14 +03:00
Cheng Su 280a2f359c [SPARK-34940][SQL][TEST] Fix test of BasicWriteTaskStatsTrackerSuite
### What changes were proposed in this pull request?

This is to fix the minor typo in unit test of BasicWriteTaskStatsTrackerSuite (https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala#L152 ), where it should be a new file name, e.g. `f-3-3`, because the unit test expects 3 files in statistics (https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala#L160 ).

### Why are the changes needed?

Fix minor bug.

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

No.

### How was this patch tested?

Changed unit test `"Three files, last one empty"` itself.

Closes #32034 from c21/tracker-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-02 15:51:22 +09:00
Max Gekk 1d084513b9 [SPARK-34938][SQL][TESTS] Benchmark only legacy interval in ExtractBenchmark
### What changes were proposed in this pull request?
In the PR, I propose to disable ANSI intervals as the result of dates/timestamp subtraction in `ExtractBenchmark` and benchmark only legacy intervals because `EXTRACT( .. FROM ..)` doesn't support ANSI intervals so far.

### Why are the changes needed?
This fixes the benchmark failure:
```
[info]   Running case: YEAR of interval
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'year((subtractdates(CAST(timestamp_seconds(id) AS DATE), DATE '0001-01-01') + subtracttimestamps(timestamp_seconds(id), TIMESTAMP '1000-01-01 01:02:03.123456')))' due to data type mismatch: argument 1 requires date type, however, '(subtractdates(CAST(timestamp_seconds(id) AS DATE), DATE '0001-01-01') + subtracttimestamps(timestamp_seconds(id), TIMESTAMP '1000-01-01 01:02:03.123456'))' is of day-time interval type.; line 1 pos 0;
[error] 'Project [extract(YEAR, (subtractdates(cast(timestamp_seconds(id#1456L) as date), 0001-01-01, false) + subtracttimestamps(timestamp_seconds(id#1456L), 1000-01-01 01:02:03.123456, false, Some(Europe/Moscow)))) AS YEAR#1458]
[error] +- Range (1262304000, 1272304000, step=1, splits=Some(1))
[error] 	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
[error] 	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:194)
```

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

### How was this patch tested?
By running the `ExtractBenchmark` benchmark via:
```
$ build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.ExtractBenchmark"
```

Closes #32035 from MaxGekk/fix-ExtractBenchmark.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-02 15:45:32 +09:00
yi.wu f897cc2374 [SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join
### What changes were proposed in this pull request?

This PR introduces a new analysis rule `DeduplicateRelations`, which deduplicates any duplicate relations in a plan first and then deduplicates conflicting attributes(which resued the `dedupRight` of `ResolveReferences`).

### Why are the changes needed?

`CostBasedJoinReorder` could fail when applying on self-join, e.g.,

```scala
// test in JoinReorderSuite
test("join reorder with self-join") {
  val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
      .select(nameToAttr("t1.v-1-10"))
      .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5")))

    // this can fail
    Optimize.execute(plan.analyze)
}
```
Besides, with the new rule `DeduplicateRelations`, we'd be able to enable some optimizations, e.g., LeftSemiAnti pushdown, redundant project removal, as reflects in updated unit tests.

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

### How was this patch tested?

Added and updated unit tests.

Closes #32027 from Ngone51/join-reorder-3.

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>
2021-04-02 06:22:57 +00:00
Cheng Su 1fc66f6870 [SPARK-34862][SQL] Support nested column in ORC vectorized reader
### What changes were proposed in this pull request?

This PR is to support nested column type in Spark ORC vectorized reader. Currently ORC vectorized reader [does not support nested column type (struct, array and map)](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala#L138). We implemented nested column vectorized reader for FB-ORC in our internal fork of Spark. We are seeing performance improvement compared to non-vectorized reader when reading nested columns. In addition, this can also help improve the non-nested column performance when reading non-nested and nested columns together in one query.

Before this PR:

* `OrcColumnVector` is the implementation class for Spark's `ColumnVector` to wrap Hive's/ORC's `ColumnVector` to read `AtomicType` data.

After this PR:

* `OrcColumnVector` is an abstract class to keep interface being shared between multiple implementation class of orc column vectors, namely `OrcAtomicColumnVector` (for `AtomicType`), `OrcArrayColumnVector` (for `ArrayType`), `OrcMapColumnVector` (for `MapType`), `OrcStructColumnVector` (for `StructType`). So the original logic to read `AtomicType` data is moved from `OrcColumnVector` to `OrcAtomicColumnVector`. The abstract class of `OrcColumnVector` is needed here because of supporting nested column (i.e. nested column vectors).
* A utility method `OrcColumnVectorUtils.toOrcColumnVector` is added to create Spark's `OrcColumnVector` from Hive's/ORC's `ColumnVector`.
* A new user-facing config `spark.sql.orc.enableNestedColumnVectorizedReader` is added to control enabling/disabling vectorized reader for nested columns. The default value is false (i.e. disabling by default). For certain tables having deep nested columns, vectorized reader might take too much memory for each sub-column vectors, compared to non-vectorized reader. So providing a config here to work around OOM for query reading wide and deep nested columns if any. We plan to enable it by default on 3.3. Leave it disable in 3.2 in case for any unknown bugs.

### Why are the changes needed?

Improve query performance when reading nested columns from ORC file format.
Tested with locally adding a small benchmark in `OrcReadBenchmark.scala`. Seeing more than 1x run time improvement.

```
Running benchmark: SQL Nested Column Scan
  Running case: Native ORC MR
  Stopped after 2 iterations, 37850 ms
  Running case: Native ORC Vectorized (Enabled Nested Column)
  Stopped after 2 iterations, 15892 ms
  Running case: Native ORC Vectorized (Disabled Nested Column)
  Stopped after 2 iterations, 37954 ms
  Running case: Hive built-in ORC
  Stopped after 2 iterations, 35118 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
SQL Nested Column Scan:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------
Native ORC MR                                           18706          18925         310          0.1       17839.6       1.0X
Native ORC Vectorized (Enabled Nested Column)            7625           7946         455          0.1        7271.6       2.5X
Native ORC Vectorized (Disabled Nested Column)          18415          18977         796          0.1       17561.5       1.0X
Hive built-in ORC                                       17469          17559         127          0.1       16660.1       1.1X
```

Benchmark:

```
nestedColumnScanBenchmark(1024 * 1024)
def nestedColumnScanBenchmark(values: Int): Unit = {
    val benchmark = new Benchmark(s"SQL Nested Column Scan", values, output = output)

    withTempPath { dir =>
      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
        import spark.implicits._
        spark.range(values).map(_ => Random.nextLong).map { x =>
          val arrayOfStructColumn = (0 until 5).map(i => (x + i, s"$x" * 5))
          val mapOfStructColumn = Map(
            s"$x" -> (x * 0.1, (x, s"$x" * 100)),
            (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)),
            (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300)))
          (arrayOfStructColumn, mapOfStructColumn)
        }.toDF("col1", "col2")
          .createOrReplaceTempView("t1")

        prepareTable(dir, spark.sql(s"SELECT * FROM t1"))

        benchmark.addCase("Native ORC MR") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Native ORC Vectorized (Enabled Nested Column)") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
        }

        benchmark.addCase("Native ORC Vectorized (Disabled Nested Column)") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Hive built-in ORC") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM hiveOrcTable").noop()
        }

        benchmark.run()
      }
    }
  }
```

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

No.

### How was this patch tested?

Added one simple test in `OrcSourceSuite.scala` to verify correctness.
Definitely need more unit tests and add benchmark here, but I want to first collect feedback before crafting more tests.

Closes #31958 from c21/orc-vector.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-01 23:10:34 -07:00
Kent Yao 1b553da2a1 [SPARK-34908][SQL][TESTS] Add test cases for char and varchar with functions
### What changes were proposed in this pull request?

Using char and varchar with the string functions and some other expressions might be confusing and ambiguous. In this PR we add test cases for char and varchar with these operations to reveal these behavior and see if we can come up with a general pattern for them.

### Why are the changes needed?

test coverage

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

no

### How was this patch tested?

new tests

Closes #32010 from yaooqinn/SPARK-34908.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-01 16:33:30 +09:00
Max Gekk 5911faa0d4 [SPARK-34903][SQL] Return day-time interval from timestamps subtraction
### What changes were proposed in this pull request?
Modify the `SubtractTimestamps` expression to return values of `DayTimeIntervalType` when `spark.sql.legacy.interval.enabled` is set to `false` (which is the default).

### Why are the changes needed?
To conform to the ANSI SQL standard which requires ANSI intervals as the result of timestamps subtraction, see
<img width="656" alt="Screenshot 2021-03-29 at 19 09 34" src="https://user-images.githubusercontent.com/1580697/112866455-7e2f0d00-90c2-11eb-96e6-3feb7eea7e09.png">

### Does this PR introduce _any_ user-facing change?
Yes.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```
and some tests from `SQLQueryTestSuite`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32016 from MaxGekk/subtract-timestamps-to-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-01 10:27:58 +03:00
ulysses-you 89ae83d19b [SPARK-34919][SQL] Change partitioning to SinglePartition if partition number is 1
### What changes were proposed in this pull request?

Change partitioning to `SinglePartition`.

### Why are the changes needed?

For node `Repartition` and `RepartitionByExpression`, if partition number is 1 we can use `SinglePartition` instead of other `Partitioning`.

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

No

### How was this patch tested?

Add test

Closes #32012 from ulysses-you/SPARK-34919.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-01 06:59:31 +00:00
Hyukjin Kwon 8a2138d09f [SPARK-34881][SQL][FOLLOW-UP] Use multiline string for TryCast' expression description
### What changes were proposed in this pull request?

This PR fixes JDK 11 compilation failed:

```
/home/runner/work/spark/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TryCast.scala:35: error: annotation argument needs to be a constant; found: "_FUNC_(expr AS type) - Casts the value `expr` to the target data type `type`. ".+("This expression is identical to CAST with configuration `spark.sql.ansi.enabled` as ").+("true, except it returns NULL instead of raising an error. Note that the behavior of this ").+("expression doesn\'t depend on configuration `spark.sql.ansi.enabled`.")
    "true, except it returns NULL instead of raising an error. Note that the behavior of this " +
```

For whatever reason, it doesn't know that the string is actually a constant. This PR simply switches it to multi-line style (which is actually more correct).

Reference:

bd0990e3e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala (L53-L57)

### Why are the changes needed?

To recover the build.

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

No, dev-only.

### How was this patch tested?

 CI in this PR

Closes #32019 from HyukjinKwon/SPARK-34881.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-01 14:50:05 +08:00
HyukjinKwon cc451c16a3 Revert "[SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join"
This reverts commit f05b940749.
2021-04-01 12:48:29 +09:00
Tanel Kiis 90f2d4d9cf [SPARK-34882][SQL] Replace if with filter clause in RewriteDistinctAggregates
### What changes were proposed in this pull request?

Replaced the `agg(if (('gid = 1)) 'cat1 else null)` pattern in `RewriteDistinctAggregates` with `agg('cat1) FILTER (WHERE 'gid = 1)`

### Why are the changes needed?

For aggregate functions, that do not ignore NULL values (`First`, `Last` or `UDAF`s) the current approach can return wrong results.

In the added UT there are no nulls in the input `testData`. The query returned `Row(0, 1, 0, 51, 100)` before this PR.

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

Bugfix

### How was this patch tested?

UT

Closes #31983 from tanelk/SPARK-34882_distinct_agg_filter.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-01 07:42:53 +09:00
Gengliang Wang 3951e3371a [SPARK-34881][SQL] New SQL Function: TRY_CAST
### What changes were proposed in this pull request?

Add a new SQL function `try_cast`.
`try_cast` is identical to  `AnsiCast` (or `Cast` when `spark.sql.ansi.enabled` is true), except it returns NULL instead of raising an error.
This expression has one major difference from `cast` with `spark.sql.ansi.enabled` as true: when the source value can't be stored in the target integral(Byte/Short/Int/Long) type, `try_cast` returns null instead of returning the low order bytes of the source value.
Note that the result of `try_cast` is not affected by the configuration `spark.sql.ansi.enabled`.

This is learned from Google BigQuery and Snowflake:
https://docs.snowflake.com/en/sql-reference/functions/try_cast.html
https://cloud.google.com/bigquery/docs/reference/standard-sql/functions-and-operators#safe_casting

### Why are the changes needed?

This is an useful for the following scenarios:
1. When ANSI mode is on, users can choose `try_cast` an alternative way to run SQL without errors for certain operations.
2. When ANSI mode is off, users can use `try_cast` to get a more reasonable result for casting a value to an integral type: when an overflow error happens, `try_cast` returns null while `cast` returns the low order bytes of the source value.

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

Yes, adding a new function `try_cast`

### How was this patch tested?

Unit tests.

Closes #31982 from gengliangwang/tryCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-31 20:47:04 +08:00
yi.wu f05b940749 [SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join
### What changes were proposed in this pull request?

This PR introduces a new analysis rule `DeduplicateRelations`, which deduplicates any duplicate relations in a plan first and then deduplicates conflicting attributes(which resued the `dedupRight` of `ResolveReferences`).

### Why are the changes needed?

`CostBasedJoinReorder` could fail when applying on self-join, e.g.,

```scala
// test in JoinReorderSuite
test("join reorder with self-join") {
  val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
      .select(nameToAttr("t1.v-1-10"))
      .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5")))

    // this can fail
    Optimize.execute(plan.analyze)
}
```
Besides, with the new rule `DeduplicateRelations`, we'd be able to enable some optimizations, e.g., LeftSemiAnti pushdown, redundant project removal, as reflects in updated unit tests.

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

### How was this patch tested?

Added and updated unit tests.

Closes #31470 from Ngone51/join-reorder.

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>
2021-03-31 14:28:35 +08:00
Angerszhuuuu eecc43cb52 [SPARK-34568][SQL] When SparkContext's conf not enable hive, we should respect enableHiveSupport() when build SparkSession too
### What changes were proposed in this pull request?
When SparkContext is initialed, if we want to start SparkSession, when we call
`SparkSession.builder.enableHiveSupport().getOrCreate()`, the SparkSession we created won't have hive support since
we have't reset existed SC's conf's `spark.sql.catalogImplementation`.
In this PR we use sharedState.conf to decide whether we should enable Hive Support.

### Why are the changes needed?
We should respect `enableHiveSupport`

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

### How was this patch tested?
Added UT

Closes #31680 from AngersZhuuuu/SPARK-34568.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 05:59:24 +00:00
Max Gekk 162f0560e6 [SPARK-34896][SQL] Return day-time interval from dates subtraction
### What changes were proposed in this pull request?
1. Add the SQL config `spark.sql.legacy.interval.enabled` which will control when Spark SQL should use `CalendarIntervalType` instead of ANSI intervals.
2. Modify the `SubtractDates` expression to return values of `DayTimeIntervalType` when `spark.sql.legacy.interval.enabled` is set to `false` (which is the default).

### Why are the changes needed?
To conform to the ANSI SQL standard which requires ANSI intervals as the result of dates subtraction, see
<img width="656" alt="Screenshot 2021-03-29 at 19 09 34" src="https://user-images.githubusercontent.com/1580697/112866455-7e2f0d00-90c2-11eb-96e6-3feb7eea7e09.png">

### Does this PR introduce _any_ user-facing change?
Yes.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```
and some tests from `SQLQueryTestSuite`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #31996 from MaxGekk/subtract-dates-to-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-31 08:20:13 +03:00
Tim Armstrong 13b255fefd [SPARK-34909][SQL] Fix conversion of negative to unsigned in conv()
### What changes were proposed in this pull request?
Use `java.lang.Long.divideUnsigned()` to do integer division in `NumberConverter` to avoid a bug in `unsignedLongDiv` that produced invalid results.

### Why are the changes needed?
The previous results are incorrect, the result of the below query should be 45012021522523134134555
```
scala> spark.sql("select conv('-10', 11, 7)").show(20, 150)
+-----------------------+
|       conv(-10, 11, 7)|
+-----------------------+
|4501202152252313413456|
+-----------------------+
scala> spark.sql("select hex(conv('-10', 11, 7))").show(20, 150)
+----------------------------------------------+
|                         hex(conv(-10, 11, 7))|
+----------------------------------------------+
|3435303132303231353232353233313334313334353600|
+----------------------------------------------+
```

### Does this PR introduce _any_ user-facing change?
`conv()` will produce different results because the bug is fixed.

### How was this patch tested?
Added a simple unit test.

Closes #32006 from timarmstrong/conv-unsigned.

Authored-by: Tim Armstrong <tim.armstrong@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 12:58:29 +08:00
Takeshi Yamamuro 46f96e9ce1 [SPARK-34795][SQL][TESTS] Adds a new job in GitHub Actions to check the output of TPC-DS queries
### What changes were proposed in this pull request?

This PR proposes to add a new job in GitHub Actions to check the output of TPC-DS queries.

NOTE: I've checked that the new job took 17m 35s in the GitHub Actions env.

### Why are the changes needed?

There are some cases where we noticed runtime-realted bugs after merging commits (e.g. .SPARK-33822). Therefore, I think it is worth adding a new job in GitHub Actions to check query output of TPC-DS (sf=1).

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

No.

### How was this patch tested?

The new test added.

Closes #31886 from maropu/TPCDSQueryTestSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-31 09:48:19 +09:00
Gengliang Wang c902f77b42 [SPARK-34856][FOLLOWUP][SQL] Remove dead code from AnsiCast.typeCheckFailureMessage
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/31954/, Array type is allowed to be cast as String type. So the customized conversion failure message branch from AnsiCast.typeCheckFailureMessage won't be reached anymore.
This PR is to remove the dead code.

### Why are the changes needed?

Code clean up.

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

No

### How was this patch tested?

Just removing dead code.

Closes #32004 from gengliangwang/SPARK-34856-followup.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-30 14:22:43 -05:00
Ali Afroozeh bd0990e3e8 [SPARK-34906] Refactor TreeNode's children handling methods into specialized traits
### What changes were proposed in this pull request?
Spark query plan node hierarchy has specialized traits (or abstract classes) for handling nodes with fixed number of children, for example `UnaryExpression`, `UnaryNode` and `UnaryExec` for representing an expression, a logical plan and a physical plan with only one child, respectively. This PR refactors the `TreeNode` hierarchy by extracting the children handling functionality into the following traits. `UnaryExpression` and other similar classes now extend the corresponding new trait:
```
trait LeafLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  override final def children: Seq[T] = Nil
}

trait UnaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def child: T
  transient override final lazy val children: Seq[T] = child :: Nil
}

trait BinaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def left: T
  def right: T
  transient override final lazy val children: Seq[T] = left :: right :: Nil
}

trait TernaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def first: T
  def second: T
  def third: T
  transient override final lazy val children: Seq[T] = first :: second :: third :: Nil
}
```

This refactoring, which is part of a bigger effort to make tree transformations in Spark more efficient, has two benefits:
- It moves the children handling methods to a single place, instead of being spread in specific subclasses, which will help the future optimizations for tree traversals.
- It allows to mix in these traits with some concrete node types that could not extend the previous classes. For example, expressions with one child that extend `AggregateFunction` cannot extend `UnaryExpression` as `AggregateFunction` defines the `foldable` method final while `UnaryExpression` defines it as non final. With the new traits, we can directly extend the concrete class from `UnaryLike` in these cases. Classes with more specific child handling will make tree traversal methods faster.

In this PR we have also updated many concrete node types to extend these traits to benefit from more specific child handling.

### Why are the changes needed?

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

### How was this patch tested?

This is a refactoring, passes existing tests.

Closes #31932 from dbaliafroozeh/FactorOutChildHandlnigIntoSeparateTraits.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-03-30 20:43:18 +02:00
ulysses-you 24d39a5ee2 [SPARK-34899][SQL] Use origin plan if we can not coalesce shuffle partition
### What changes were proposed in this pull request?

Add check if `CoalesceShufflePartitions` really coalesce shuffle partition number.

### Why are the changes needed?

The `CoalesceShufflePartitions` can not coalesce such case if the total shuffle partitions size of mappers are big enough. Then it's confused to use `CustomShuffleReaderExec` which marked as `coalesced` but has no affect with partition number.

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

Probably yes, the plan changed.

### How was this patch tested?

Add test.

Closes #31994 from ulysses-you/SPARK-34899.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 13:50:19 +00:00
Yuming Wang de66fa63f9 [SPARK-34884][SQL] Improve DPP evaluation to make filtering side must can broadcast by size or broadcast by hint
### What changes were proposed in this pull request?

Improve dynamic partition pruning evaluation to make filtering side must can broadcast by size or broadcast by hint.

### Why are the changes needed?

1. Fast fail if filtering side can not broadcast by size or broadcast by hint.
2. We can safely disable `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly`.

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

No.

### How was this patch tested?

Existing unit test.

Closes #31984 from wangyum/SPARK-34884.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:34:46 +00:00
angerszhu a98dc60408 [SPARK-33308][SQL] Refactor current grouping analytics
### What changes were proposed in this pull request?
As discussed in
https://github.com/apache/spark/pull/30145#discussion_r514728642
https://github.com/apache/spark/pull/30145#discussion_r514734648

We need to rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.
In  postgres sql, it support
```
select a, b, c, count(1) from t group by cube (a, b, c);
select a, b, c, count(1) from t group by cube(a, b, c);
select a, b, c, count(1) from t group by cube (a, b, c, (a, b), (a, b, c));
select a, b, c, count(1) from t group by rollup(a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c, (a, b), (a, b, c));
```
In this pr,  we have done three things as below, and we will split it to different pr:

 - Refactor CUBE/ROLLUP (regarding them as ANTLR tokens in a parser)
 - Refactor GROUPING SETS (the logical node -> a new expr)
 - Support new syntax for CUBE/ROLLUP (e.g., GROUP BY CUBE ((a, b), (a, c)))

### Why are the changes needed?
Rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.

### Does this PR introduce _any_ user-facing change?
User can  write Grouping Analytics grammar as flexible as Postgres SQL to support subsequent development.

### How was this patch tested?
Added UT

Closes #30212 from AngersZhuuuu/refact-grouping-analytics.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:31:58 +00:00
Cheng Su 935aa8c8db [SPARK-32985][SQL][FOLLOWUP] Rename createNonBucketedReadRDD and minor change in FileSourceScanExec
### What changes were proposed in this pull request?

This PR is a followup change to address comments in https://github.com/apache/spark/pull/31413#discussion_r603280965 and https://github.com/apache/spark/pull/31413#discussion_r603296475 . Minor change in `FileSourceScanExec`. No actual logic change here.

### Why are the changes needed?

Better readability.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32000 from c21/bucket-scan.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 19:57:32 +09:00
David Li 1237124062 [SPARK-34463][PYSPARK][DOCS] Document caveats of Arrow selfDestruct
### What changes were proposed in this pull request?

As a followup for #29818, document caveats of using the Arrow selfDestruct option in toPandas, which include:
- toPandas() may be slower;
- the resulting dataframe may not support some Pandas operations due to immutable backing arrays.

### Why are the changes needed?

This will hopefully reduce user confusion as with SPARK-34463.

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

Yes - documentation is updated and a config setting description is updated to clearly indicate the config is experimental.

### How was this patch tested?
This is a documentation-only change.

Closes #31738 from lidavidm/spark-34463.

Authored-by: David Li <li.davidm96@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 13:30:27 +09:00
yangjie01 7158e7f986 [SPARK-34900][TEST] Make sure benchmarks can run using spark-submit cmd described in the guide
### What changes were proposed in this pull request?
Some `spark-submit`  commands used to run benchmarks in the user's guide is wrong, we can't use these commands to run benchmarks successful.

So the major changes of this pr is correct these wrong commands, for example, run a benchmark which inherits from `SqlBasedBenchmark`, we must specify `--jars <spark core test jar>,<spark catalyst test jar>` because `SqlBasedBenchmark` based benchmark extends `BenchmarkBase(defined in spark core test jar)` and `SQLHelper(defined in spark catalyst test jar)`.

Another change of this pr is removed the `scalatest Assertions` dependency of Benchmarks because `scalatest-*.jar` are not in the distribution package, it will be troublesome to use.

### Why are the changes needed?
Make sure benchmarks can run using spark-submit cmd described in the guide

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

### How was this patch tested?
Use the corrected `spark-submit` commands to run benchmarks successfully.

Closes #31995 from LuciferYang/fix-benchmark-guide.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 11:58:01 +09:00
Yuming Wang fcef2375a3 [SPARK-34622][SQL] Push down limit through Project with Join
### What changes were proposed in this pull request?

There is a `Project` between `LocalLimit` and `Join` if `Join`'s output do not match the `LocalLimit`'s output. This pr add support push down limit through this case. For example:
   ```scala
   spark.sql("create table t1(a int, b int, c int) using parquet")
   spark.sql("create table t2(x int, y int, z int) using parquet")
   spark.sql("select a from t1 left join t2 on a = x and b = y limit 5").explain("extended")
   ```

   ```
   == Optimized Logical Plan ==
   GlobalLimit 5
   +- LocalLimit 5
      +- Project [a#0]
         +- Join LeftOuter, ((a#0 = x#3) AND (b#1 = y#4))
            :- Project [a#0, b#1]
            :  +- Relation default.t1[a#0,b#1,c#2] parquet
            +- Project [x#3, y#4]
               +- Filter (isnotnull(x#3) AND isnotnull(y#4))
                  +- Relation default.t2[x#3,y#4,z#5] parquet
   ```

   After this pr:
   ```
   == Optimized Logical Plan ==
   GlobalLimit 5
   +- LocalLimit 5
      +- Project [a#0]
         +- Join LeftOuter, ((a#0 = x#3) AND (b#1 = y#4))
            :- LocalLimit 5
            :  +- Project [a#0, b#1]
            :     +- Relation default.t1[a#0,b#1,c#2] parquet
            +- Project [x#3, y#4]
               +- Filter (isnotnull(x#3) AND isnotnull(y#4))
                  +- Relation default.t2[x#3,y#4,z#5] parquet
   ```

### Why are the changes needed?

Improve limit push down to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31739 from wangyum/SPARK-34622.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-30 10:45:30 +09:00
Jungtaek Lim 43e08b1f0f [SPARK-34255][SQL] Support partitioning with static number on required distribution and ordering on V2 write
### What changes were proposed in this pull request?

This PR proposes to extend the functionality of requirement for distribution and ordering on V2 write to specify the number of partitioning on repartition, so that data source is able to control the parallelism and determine the data distribution per partition in prior.

The partitioning with static number is optional, and by default disabled via default method, so only implementations required to restrict the number of partition statically need to override the method and provide the number.

Note that we don't support static number of partitions with unspecified distribution for this PR, as we haven't found the real use cases, and for hypothetical case the static number isn't good enough. Javadoc clearly describes the limitation.

### Why are the changes needed?

The use case comes from feature parity with DSv1.

I have state data source which enables the state in SS to be rewritten, which enables repartitioning, schema evolution, etc via batch query. The writer requires hash partitioning against group key, with the "desired number of partitions", which is same as what Spark does read and write against state.

This is now implemented as DSv1, and the requirement is simply done by calling repartition with the "desired number".

```
val fullPathsForKeyColumns = keySchema.map(key => new Column(s"key.${key.name}"))
data
  .repartition(newPartitions, fullPathsForKeyColumns: _*)
  .queryExecution
  .toRdd
  .foreachPartition(
    writeFn(resolvedCpLocation, version, operatorId, storeName, keySchema, valueSchema,
      storeConf, hadoopConfBroadcast, queryId))
```

Thanks to SPARK-34026, it's now possible to require the hash partitioning, but still not able to require the number of partitions. This PR will enable to let data source require the number of partitions.

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

Yes, but only for data source implementors. Even for them, this is no breaking change as default method is added.

### How was this patch tested?

Added UTs.

Closes #31355 from HeartSaVioR/SPARK-34255.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 14:33:23 +00:00
Kousuke Saruta 14c7bb877d [SPARK-34872][SQL] quoteIfNeeded should quote a name which contains non-word characters
### What changes were proposed in this pull request?

This PR fixes an issue that `quoteIfNeeded` quotes a name only if it contains `.` or ``` ` ```.
This method should quote it if it contains non-word characters.

### Why are the changes needed?

It's a potential bug.

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

No.

### How was this patch tested?

New test.

Closes #31964 from sarutak/fix-quoteIfNeeded.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 09:31:24 +00:00
Angerszhuuuu 015c59843c [SPARK-34879][SQL] HiveInspector supports DayTimeIntervalType and YearMonthIntervalType
### What changes were proposed in this pull request?
Make HiveInspector support DayTimeIntervalType and YearMonthIntervalType.
Then we can use these two types in HiveUDF and HiveScriptTransformation

### Why are the changes needed?
Support more data type when use hive serde

### Does this PR introduce _any_ user-facing change?
User can use  `DayTimeIntervalType` and `YearMonthIntervalType` in HiveUDF and  HiveScriptTransformation

### How was this patch tested?
Added UT

Closes #31979 from AngersZhuuuu/SPARK-34879.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-29 08:38:20 +03:00
Angerszhuuuu 2356cdd420 [SPARK-34814][SQL] LikeSimplification should handle NULL
### What changes were proposed in this pull request?
LikeSimplification should handle NULL.

UT will failed  before this pr
```
  test("SPARK-34814: LikeSimplification should handle NULL") {
    withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key ->
      ConstantFolding.getClass.getName.stripSuffix("$")) {
      checkEvaluation(Literal.create("foo", StringType)
        .likeAll("%foo%", Literal.create(null, StringType)), null)
    }
  }

[info] - test *** FAILED *** (2 seconds, 443 milliseconds)
[info]   java.lang.NullPointerException:
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$.$anonfun$simplifyMultiLike$1(expressions.scala:697)
[info]   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[info]   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
[info]   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
[info]   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
[info]   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$.org$apache$spark$sql$catalyst$optimizer$LikeSimplification$$simplifyMultiLike(expressions.scala:697)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$$anonfun$apply$9.applyOrElse(expressions.scala:722)
[info]   at org.apache.spark.sql.catalyst.optimizer.LikeSimplification$$anonfun$apply$9.applyOrElse(expressions.scala:714)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:316)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:316)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:321)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:406)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:242)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:404)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:357)
[info]   at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:321)
[info]   at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsDown$1(QueryPlan.scala:94)
[info]   at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:116)
[info]   at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72)
```

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #31976 from AngersZhuuuu/SPARK-34814.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-29 12:05:00 +09:00
Tanel Kiis 4b9e94c444 [SPARK-34876][SQL] Fill defaultResult of non-nullable aggregates
### What changes were proposed in this pull request?

Filled the `defaultResult` field on non-nullable aggregates

### Why are the changes needed?

The `defaultResult` defaults to `None` and in some situations (like correlated scalar subqueries) it is used for the value of the aggregation.

The UT result before the fix:
```
-- !query
SELECT t1a,
   (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2,
   (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2,
   (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2,
   (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2,
   (SELECT collect_set(t2d) FROM t2 WHERE t2a = t1a) collect_set_t2,
    (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2
FROM t1
-- !query schema
struct<t1a:string,count_t2:bigint,count_if_t2:bigint,approx_count_distinct_t2:bigint,collect_list_t2:array<bigint>,collect_set_t2:array<bigint>,collect_set_t2:string>
-- !query output
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1b	6	6	3	[19,119,319,19,19,19]	[19,119,319]	0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001
val1c	2	2	2	[219,19]	[219,19]	0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
```

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

Bugfix

### How was this patch tested?

UT

Closes #31973 from tanelk/SPARK-34876_non_nullable_agg_subquery.

Authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-29 11:47:08 +09:00
hanover-fiste 4fceef0159 [SPARK-34843][SQL] Calculate more precise partition stride in JDBCRelation
### What changes were proposed in this pull request?
The changes being proposed are to increase the accuracy of JDBCRelation's stride calculation, as outlined in: https://issues.apache.org/jira/browse/SPARK-34843

In summary:

Currently, in JDBCRelation (line 123), the stride size is calculated as follows:
val stride: Long = upperBound / numPartitions - lowerBound / numPartitions

Due to truncation happening on both divisions, the stride size can fall short of what it should be. This can lead to a big difference between the provided upper bound and the actual start of the last partition.

I'm proposing a different formula that doesn't truncate to early, and also maintains accuracy using fixed-point decimals. This helps tremendously with the size of the last partition, which can be even more amplified if there is data skew in that direction. In a real-life test, I've seen a 27% increase in performance with this more proper stride alignment. The reason for fixed-point decimals instead of floating-point decimals is because inaccuracy due to limitation of what the float can represent. This may seem small, but could shift the midpoint a bit, and depending on how granular the data is, that could translate to quite a difference. It's also just inaccurate, and I'm striving to make the partitioning as accurate as possible, within reason.

Lastly, since the last partition's predicate is determined by how the strides align starting from the lower bound (plus one stride), there can be skew introduced creating a larger last partition compared to the first partition. Therefore, after calculating a more precise stride size, I've also introduced logic to move the first partition's predicate (which is an offset from the lower bound) to a position that closely matches the offset of the last partition's predicate (in relation to the upper bound). This makes the first and last partition more evenly distributed compared to each other, and helps with the last task being the largest (reducing its size).

### Why are the changes needed?
The current implementation is inaccurate and can lead to the last task/partition running much longer than previous tasks. Therefore, you can end up with a single node/core running for an extended period while other nodes/cores are sitting idle.

### Does this PR introduce _any_ user-facing change?
No. I would suspect some users will just get a good performance increase. As stated above, if we were to run our code on Spark that has this change implemented, we would have all of the sudden got a 27% increase in performance.

### How was this patch tested?
I've added two new unit tests. I did need to update one unit test, but when you look at the comparison of the before and after, you'll see better alignment of the partitioning with the new implementation. Given that the lower partition's predicate is exclusive and the upper's is inclusive, the offset of the lower was 3 days, and the offset of the upper was 6 days... that's potentially twice the amount of data in that upper partition (could be much more depending on how the user's data is distributed).

Other unit tests that utilize timestamps and two partitions have maintained their midpoint.

### Examples

I've added results with and without the realignment logic to better highlight both improvements this PR brings.

**Example 1:**
Given the following partition config:
"lowerBound" -> "1930-01-01"
"upperBound" -> "2020-12-31"
"numPartitions" -> 1000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1930-02-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-07-11'
_Old method, but with new realingment logic of first partition:_
First partition: "PartitionColumn" < '1931-10-14' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2019-03-22'

_New method:_
First partition: "PartitionColumn" < '1930-02-03' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-05'
_New with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1930-06-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-08-02'

**Example 2:**
Given the following partition config:
"lowerBound" -> "1927-04-05",
"upperBound" -> "2020-10-16"
"numPartitions" -> 2000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1927-04-21' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2014-10-29'
_Old method, but with new realingment logic of first partition::_
First partition: "PartitionColumn" < '1930-04-07' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-10-15'

_New method:_
First partition: "PartitionColumn" < '1927-04-22' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-19'
_New method with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1927-07-13' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-07-10'

Closes #31965 from hanover-fiste/SPARK-34843.

Authored-by: hanover-fiste <jyarbrough.git@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-28 12:59:20 -05:00
Peter Toth 3382190349 [SPARK-34829][SQL] Fix higher order function results
### What changes were proposed in this pull request?
This PR fixes a correctness issue with higher order functions. The results of function expressions needs to be copied in some higher order functions as such an expression can return with internal buffers and higher order functions can call multiple times the expression.
The issue was discovered with typed `ScalaUDF`s after https://github.com/apache/spark/pull/28979.

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

### Does this PR introduce _any_ user-facing change?
Yes, some queries return the right results again.

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

Closes #31955 from peter-toth/SPARK-34829-fix-scalaudf-resultconversion.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 10:01:09 -07:00
Yuming Wang 540f1fb1d9 [SPARK-32855][SQL][FOLLOWUP] Fix code format in SQLConf and comment in PartitionPruning
### What changes were proposed in this pull request?

Fix code format in `SQLConf` and comment in `PartitionPruning`.

### Why are the changes needed?

Make code more readable.

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

No.

### How was this patch tested?

N/A

Closes #31969 from wangyum/SPARK-32855-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 09:48:54 -07:00
Dongjoon Hyun e7af44861e [SPARK-34880][SQL][TESTS] Add Parquet ZSTD compression test coverage
### What changes were proposed in this pull request?

Apache Parquet 1.12.0 switches its ZSTD compression from Hadoop codec to its own codec.

### Why are the changes needed?

**Apache Spark 3.1 (It requires libhadoop built with zstd)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
21/03/27 08:49:38 ERROR Executor: Exception in task 11.0 in stage 0.0 (TID 11)2]
java.lang.RuntimeException: native zStandard library not available:
this version of libhadoop was built without zstd support.
```

**Apache Spark 3.2 (No libhadoop requirement)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
```

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

Yes, this is an improvement.

### How was this patch tested?

Pass the CI with the newly added test coverage.

Closes #31981 from dongjoon-hyun/SPARK-34880.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 12:48:12 -07:00
Yuming Wang cbffc12f90 [SPARK-34542][BUILD] Upgrade Parquet to 1.12.0
### What changes were proposed in this pull request?

Parquet 1.12.0 New Feature
- PARQUET-41 - Add bloom filters to parquet statistics
- PARQUET-1373 - Encryption key management tools
- PARQUET-1396 - Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory
- PARQUET-1622 - Add BYTE_STREAM_SPLIT encoding
- PARQUET-1784 - Column-wise configuration
- PARQUET-1817 - Crypto Properties Factory
- PARQUET-1854 - Properties-Driven Interface to Parquet Encryption

Parquet 1.12.0 release notes:
https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/CHANGES.md

### Why are the changes needed?

- Bloom filters to improve filter performance
- ZSTD enhancement

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

No.

### How was this patch tested?

Existing unit test.

Closes #31649 from wangyum/SPARK-34542.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <yumwang@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 07:56:29 -07:00
Angerszhuuuu 468b944b00 [SPARK-34841][SQL] Push ANSI interval binary expressions into into (if/else) branches
### What changes were proposed in this pull request?
Push ANSI interval binary expressions into into (if / case) branches

### Why are the changes needed?
Support more binary expression to push into if/else and casewhen

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

### How was this patch tested?
Added UT

Closes #31978 from AngersZhuuuu/SPARK-34841.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-27 14:50:28 +03:00
Angerszhuuuu 769cf7b966 [SPARK-34744][SQL] Improve error message for casting cause overflow error
### What changes were proposed in this pull request?
Improve error message for casting cause overflow error. We should use DataType's catalogString.

### Why are the changes needed?
Improve error message

### Does this PR introduce _any_ user-facing change?
For example:
```
set spark.sql.ansi.enabled=true;
select tinyint(128) * tinyint(2);
```
Error message before this pr:
```
Casting 128 to scala.Byte$ causes overflow
```
After this pr:
```
Casting 128 to tinyint causes overflow
```

### How was this patch tested?
Added UT

Closes #31971 from AngersZhuuuu/SPARK-34744.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-27 11:15:55 +08:00
Max Gekk 9ba889b6ea [SPARK-34875][SQL] Support divide a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `day-time interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31972 from MaxGekk/div-dt-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 15:36:08 +00:00
Wenchen Fan 61d038f26e Revert "[SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand"
This reverts commit da04f1f4f8.
2021-03-26 15:26:48 +08:00
Max Gekk f212c61c43 [SPARK-34868][SQL] Support divide an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `year-month interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31961 from MaxGekk/div-ym-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 05:56:56 +00:00
Yuming Wang aaa0d2a66b [SPARK-32855][SQL] Improve the cost model in pruningHasBenefit for filtering side can not build broadcast by join type
### What changes were proposed in this pull request?

This pr improve the cost model in `pruningHasBenefit` for filtering side can not build broadcast by join type:
1. The filtering side must be small enough to build broadcast by size.
2. The estimated size of the pruning side must be big enough: `estimatePruningSideSize * spark.sql.optimizer.dynamicPartitionPruning.pruningSideExtraFilterRatio > overhead`.

### Why are the changes needed?

Improve query performance for these cases.

This a real case from cluster. Left join and left size very small and right side can build DPP:
![image](https://user-images.githubusercontent.com/5399861/92882197-445a2a00-f442-11ea-955d-16a7724e535b.png)

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

No.

### How was this patch tested?

Unit test.

Closes #29726 from wangyum/SPARK-32855.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 04:48:13 +00:00
Kent Yao 820b465886 [SPARK-34786][SQL] Read Parquet unsigned int64 logical type that stored as signed int64 physical type to decimal(20, 0)
### What changes were proposed in this pull request?

A companion PR for SPARK-34817, when we handle the unsigned int(<=32) logical types. In this PR, we map the unsigned int64 to decimal(20, 0) for better compatibility.

### Why are the changes needed?

Spark won't have unsigned types, but spark should be able to read existing parquet files written by other systems that support unsigned types for better compatibility.

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

yes, we can read parquet uint64 now

### How was this patch tested?

new unit tests

Closes #31960 from yaooqinn/SPARK-34786-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-26 09:54:19 +08:00
Yuanjian Li 5ffc3897e0 [SPARK-34871][SS] Move the checkpoint location resolving into the rule ResolveWriteToStream
### What changes were proposed in this pull request?
Move the checkpoint location resolving into the rule ResolveWriteToStream, which is added in SPARK-34748.

### Why are the changes needed?
After SPARK-34748, we have a rule ResolveWriteToStream for the analysis logic for the resolving logic of stream write plans. Based on it, we can further move the checkpoint location resolving work in the rule. Then, all the checkpoint resolving logic was done in the analyzer.

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

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

Closes #31963 from xuanyuanking/SPARK-34871.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-26 10:29:50 +09:00
Wenchen Fan 658e95c345 [SPARK-34833][SQL][FOLLOWUP] Handle outer references in all the places
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31940 . This PR generalizes the matching of attributes and outer references, so that outer references are handled everywhere.

Note that, currently correlated subquery has a lot of limitations in Spark, and the newly covered cases are not possible to happen. So this PR is a code refactor.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31959 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-26 09:10:03 +09:00
Gengliang Wang 0515f49018 [SPARK-34856][SQL] ANSI mode: Allow casting complex types as string type
### What changes were proposed in this pull request?

Allow casting complex types as string type in ANSI mode.

### Why are the changes needed?

Currently, complex types are not allowed to cast as string type. This breaks the DataFrame.show() API. E.g
```
scala> sql(“select array(1, 2, 2)“).show(false)
org.apache.spark.sql.AnalysisException: cannot resolve ‘CAST(`array(1, 2, 2)` AS STRING)’ due to data type mismatch:
 cannot cast array<int> to string with ANSI mode on.
```
We should allow the conversion as the extension of the ANSI SQL standard, so that the DataFrame.show() still work in ANSI mode.
### Does this PR introduce _any_ user-facing change?

Yes, casting complex types as string type is now allowed in ANSI mode.

### How was this patch tested?

Unit tests.

Closes #31954 from gengliangwang/fixExplicitCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-26 00:17:43 +08:00
Karen Feng 0d91f9c3f3 [SPARK-33600][SQL] Group exception messages in execution/datasources/v2
### What changes were proposed in this pull request?

This PR groups exception messages in `execution/datasources/v2`.

### Why are the changes needed?

It will largely help with standardization of error messages and its maintenance.

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

No. Error messages remain unchanged.

### How was this patch tested?

No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31619 from karenfeng/spark-33600.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 16:15:30 +00:00
Tanel Kiis 6ba8445ea3 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

This is resubmition of #31927. The schema for one of the TPCDS tables was updated and that changed the `explain.txt` for the q17.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31957 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 10:22:49 +00:00
Tim Armstrong 1d6acd584a [SPARK-34857][SQL] Correct AtLeastNNonNulls's explain output
### What changes were proposed in this pull request?
Removed the custom toString implementation of AtLeastNNoneNulls.

### Why are the changes needed?
It shows up wrong in the explain plan. The name of the function is wrong and the actual value of the first argument is not shown. Both of these would make it easier to understand the plan.

```
(12) Filter
Input [3]: [c1#2410L, c2#2419, c3#2422]
Condition : AtLeastNNulls(n, c1#2410L)
```

### Does this PR introduce _any_ user-facing change?
Only the explain plan changes if this function is used.

### How was this patch tested?
Added a simple unit test to make sure that the toString output is correct.

Closes #31956 from timarmstrong/atleastnnonnulls.

Authored-by: Tim Armstrong <tim.armstrong@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-25 17:20:01 +09:00
Max Gekk a68d7ca8c5 [SPARK-34850][SQL] Support multiply a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * day-time interval` and `day-time interval * numeric`.
3. Invoke `DoubleMath.roundToInt` in `double/float * year-month interval`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31951 from MaxGekk/mul-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-25 10:46:50 +03:00
Kent Yao 8c6748f691 [SPARK-34817][SQL] Read parquet unsigned types that stored as int32 physical type in parquet
### What changes were proposed in this pull request?

Unsigned types may be used to produce smaller in-memory representations of the data. These types used by frameworks(e.g. hive, pig) using parquet. And parquet will map them to its base types.

see more https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift

```thrift
  /**
   * An unsigned integer value.
   *
   * The number describes the maximum number of meaningful data bits in
   * the stored value. 8, 16 and 32 bit values are stored using the
   * INT32 physical type.  64 bit values are stored using the INT64
   * physical type.
   *
   */
  UINT_8 = 11;
  UINT_16 = 12;
  UINT_32 = 13;
  UINT_64 = 14;
```

```
UInt8-[0:255]
UInt16-[0:65535]
UInt32-[0:4294967295]
UInt64-[0:18446744073709551615]
```

In this PR, we support read UINT_8 as ShortType, UINT_16 as IntegerType, UINT_32 as LongType to fit their range. Support for UINT_64 will be in another PR.

### Why are the changes needed?

better parquet support

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

yes, we can read unit[8/16/32] from parquet files

### How was this patch tested?

new tests

Closes #31921 from yaooqinn/SPARK-34817.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:58:06 +00:00
Terry Kim da04f1f4f8 [SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand
### What changes were proposed in this pull request?

This PR proposes to remove re-analyzing the already analyzed plan for `CreateViewCommand` as discussed https://github.com/apache/spark/pull/31273/files#r581592786.

### Why are the changes needed?

No need to analyze the plan if it's already analyzed.

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

No.

### How was this patch tested?

Existing tests should cover this.

Closes #31933 from imback82/remove_analyzed_from_create_temp_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:53:59 +00:00
HyukjinKwon 7838f55ca7 Revert "[SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes"
This reverts commit 84df54b495.
2021-03-25 12:31:08 +09:00
ulysses-you 9d561e6b5e [SPARK-34852][SQL] Close Hive session state should use withHiveState
### What changes were proposed in this pull request?

Wrap Hive sessionStae `close` with `withHiveState`

### Why are the changes needed?

Some reason:

1. Shutdown hook is invoked using different thread
2. Hive may use metasotre client again during closing

Otherwise, we may get such expcetion with custom hive metastore version
```
21/03/24 13:26:18 INFO session.SessionState: Failed to remove classloaders from DataNucleus
java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1654)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:80)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:130)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:101)
	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3367)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3406)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3386)
	at org.apache.hadoop.hive.ql.session.SessionState.unCacheDataNucleusClassLoaders(SessionState.java:1546)
	at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1536)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
```

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

No, since this not released.

### How was this patch tested?

manual test.

Closes #31949 from ulysses-you/SPARK-34852.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-25 10:21:44 +08:00
Takeshi Yamamuro 150769bced [SPARK-34833][SQL] Apply right-padding correctly for correlated subqueries
### What changes were proposed in this pull request?

This PR intends to fix the bug that does not apply right-padding for char types inside correlated subquries.
For example,  a query below returns nothing in master, but a correct result is `c`.
```
scala> sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING parquet")
scala> sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(7)) USING parquet")
scala> sql("INSERT INTO t1 VALUES ('c', 'b')")
scala> sql("INSERT INTO t2 VALUES ('a', 'b')")
scala> val df = sql("""
  |SELECT v FROM t1
  |WHERE 'a' IN (SELECT v FROM t2 WHERE t2.c = t1.c )""".stripMargin)

scala> df.show()
+---+
|  v|
+---+
+---+

```

This is because `ApplyCharTypePadding`  does not handle the case above to apply right-padding into `'abc'`. This PR modifies the code in `ApplyCharTypePadding` for handling it correctly.

```
// Before this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#13]
+- Filter a IN (list#12 [c#14])
   :  +- Project [v#15]
   :     +- Filter (c#16 = outer(c#14))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#15,c#16] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#13,c#14] parquet

scala> df.show()
+---+
|  v|
+---+
+---+

// After this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#43]
+- Filter a IN (list#42 [c#44])
   :  +- Project [v#45]
   :     +- Filter (c#46 = rpad(outer(c#44), 7,  ))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#45,c#46] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#43,c#44] parquet

scala> df.show()
+---+
|  v|
+---+
|  c|
+---+
```

This fix is lated to TPCDS q17; the query returns nothing because of this bug: https://github.com/apache/spark/pull/31886/files#r599333799

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Unit tests added.

Closes #31940 from maropu/FixCharPadding.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-25 08:31:57 +09:00
Gengliang Wang abfd9b23cd [SPARK-34769][SQL] AnsiTypeCoercion: return closest convertible type among TypeCollection
### What changes were proposed in this pull request?

Currently, when implicit casting a data type to a `TypeCollection`, Spark returns the first convertible data type among `TypeCollection`.
In ANSI mode, we can make the behavior more reasonable by returning the closet convertible data type in `TypeCollection`.

In details, we first try to find the all the expected types we can implicitly cast:
1. if there is no convertible data types, return None;
2. if there is only one convertible data type, cast input as it;
3. otherwise if there are multiple convertible data types, find the closet data
type among them. If there is no such closet data type, return None.

Note that if the closet type is Float type and the convertible types contains Double type, simply return Double type as the closet type to avoid potential
precision loss on converting the Integral type as Float type.

### Why are the changes needed?

Make the type coercion rule for TypeCollection more reasonable and ANSI compatible.
E.g. returning Long instead of Double for`implicast(int, TypeCollect(Double, Long))`.

From ANSI SQL Spec section 4.33 "SQL-invoked routines"
![Screen Shot 2021-03-17 at 4 05 06 PM](https://user-images.githubusercontent.com/1097932/111434916-5e104e80-86bd-11eb-8b3b-33090a68067d.png)

Section 9.6 "Subject routine determination"
![Screen Shot 2021-03-17 at 1 36 55 PM](https://user-images.githubusercontent.com/1097932/111420336-48445e80-86a8-11eb-9d50-34b325043bdb.png)

Section 10.4 "routine invocation"
![Screen Shot 2021-03-17 at 4 08 41 PM](https://user-images.githubusercontent.com/1097932/111434926-610b3f00-86bd-11eb-8c32-8c7935e055da.png)

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

Yes, in ANSI mode, implicit casting to a `TypeCollection` returns the narrowest convertible data type instead of the first convertible one.

### How was this patch tested?

Unit tests.

Closes #31859 from gengliangwang/implicitCastTypeCollection.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 15:04:03 +00:00
Tanel Kiis 84df54b495 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31927 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 14:36:51 +00:00
Cheng Su 35c70e417d [SPARK-34853][SQL] Remove duplicated definition of output partitioning/ordering for limit operator
### What changes were proposed in this pull request?

Both local limit and global limit define the output partitioning and output ordering in the same way and this is duplicated (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L159-L175 ). We can move the output partitioning and ordering into their parent trait - `BaseLimitExec`. This is doable as `BaseLimitExec` has no more other child class. This is a minor code refactoring.

### Why are the changes needed?

Clean up the code a little bit. Better readability.

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

No.

### How was this patch tested?

Pure refactoring. Rely on existing unit tests.

Closes #31950 from c21/limit-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-24 23:06:35 +09:00
yangjie01 712a62ca82 [SPARK-34832][SQL][TEST] Set EXECUTOR_ALLOW_SPARK_CONTEXT to true to ensure ExternalAppendOnlyUnsafeRowArrayBenchmark run successfully
### What changes were proposed in this pull request?
SPARK-32160 add a config(`EXECUTOR_ALLOW_SPARK_CONTEXT`) to switch allow/disallow to create `SparkContext` in executors and the default value of the config is `false`

`ExternalAppendOnlyUnsafeRowArrayBenchmark` will run fail when `EXECUTOR_ALLOW_SPARK_CONTEXT` use the default value because the `ExternalAppendOnlyUnsafeRowArrayBenchmark#withFakeTaskContext` method try to create a `SparkContext` manually in Executor Side.

So the main change of this pr is  set `EXECUTOR_ALLOW_SPARK_CONTEXT` to `true` to ensure `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

### Why are the changes needed?
Bug fix.

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

### How was this patch tested?
Manual test:
```
bin/spark-submit --class org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark --jars spark-core_2.12-3.2.0-SNAPSHOT-tests.jar spark-sql_2.12-3.2.0-SNAPSHOT-tests.jar
```

**Before**
```
Exception in thread "main" java.lang.IllegalStateException: SparkContext should only be created and accessed on the driver.
	at org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$assertOnDriver(SparkContext.scala:2679)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:89)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:137)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.withFakeTaskContext(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:52)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.testAgainstRawArrayBuffer(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:119)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.$anonfun$runBenchmarkSuite$1(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:189)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.benchmark.BenchmarkBase.runBenchmark(BenchmarkBase.scala:40)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.runBenchmarkSuite(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:186)
	at org.apache.spark.benchmark.BenchmarkBase.main(BenchmarkBase.scala:58)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark.main(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

**After**

`ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

Closes #31939 from LuciferYang/SPARK-34832.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-24 14:59:31 +09:00
Kousuke Saruta f7e9b6efc7 [SPARK-34763][SQL] col(), $"<name>" and df("name") should handle quoted column names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `col()`, `$"<name>"` and `df("name")` don't handle quoted column names  like ``` `a``b.c` ```properly.

For example, if we have a following DataFrame.
```
val df1 = spark.sql("SELECT 'col1' AS `a``b.c`")
```

For the DataFrame, this query is successfully executed.
```
scala> df1.selectExpr("`a``b.c`").show
+-----+
|a`b.c|
+-----+
| col1|
+-----+
```

But the following query will fail because ``` df1("`a``b.c`") ``` throws an exception.
```
scala> df1.select(df1("`a``b.c`")).show
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `a``b.c`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:152)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:162)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221)
  at org.apache.spark.sql.Dataset.col(Dataset.scala:1274)
  at org.apache.spark.sql.Dataset.apply(Dataset.scala:1241)
  ... 49 elided
```
### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

New tests.

Closes #31854 from sarutak/fix-parseAttributeName.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 13:34:10 +08:00
Takeshi Yamamuro 0494dc90af [SPARK-34842][SQL][TESTS] Corrects the type of date_dim.d_quarter_name in the TPCDS schema
### What changes were proposed in this pull request?

SPARK-34842 (#31012) has a typo in the type of `date_dim.d_quarter_name` in the TPCDS schema (`TPCDSBase`). This PR replace `CHAR(1)` with `CHAR(6)`. This fix comes from p28 in [the TPCDS official doc](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

N/A

Closes #31943 from maropu/SPARK-34083-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-23 10:22:13 -07:00
Max Gekk 760556a42f [SPARK-34824][SQL] Support multiply an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31929 from MaxGekk/interval-mul-div.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-23 19:40:15 +03:00
Wenchen Fan 3b70829b5b [SPARK-34719][SQL] Correctly resolve the view query with duplicated column names
forward-port https://github.com/apache/spark/pull/31811 to master

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

For permanent views (and the new SQL temp view in Spark 3.1), we store the view SQL text and re-parse/analyze the view SQL text when reading the view. In the case of `SELECT * FROM ...`, we want to avoid view schema change (e.g. the referenced table changes its schema) and will record the view query output column names when creating the view, so that when reading the view we can add a `SELECT recorded_column_names FROM ...` to retain the original view query schema.

In Spark 3.1 and before, the final SELECT is added after the analysis phase: https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala#L67

If the view query has duplicated output column names, we always pick the first column when reading a view. A simple repro:
```
scala> sql("create view c(x, y) as select 1 a, 2 a")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("select * from c").show
+---+---+
|  x|  y|
+---+---+
|  1|  1|
+---+---+
```

In the master branch, we will fail at the view reading time due to b891862fb6 , which adds the final SELECT during analysis, so that the query fails with `Reference 'a' is ambiguous`

This PR proposes to resolve the view query output column names from the matching attributes by ordinal.

For example,  `create view c(x, y) as select 1 a, 2 a`, the view query output column names are `[a, a]`. When we reading the view, there are 2 matching attributes (e.g.`[a#1, a#2]`) and we can simply match them by ordinal.

A negative example is
```
create table t(a int)
create view v as select *, 1 as col from t
replace table t(a int, col int)
```
When reading the view, the view query output column names are `[a, col]`, and there are two matching attributes of `col`, and we should fail the query. See the tests for details.

### Why are the changes needed?

bug fix

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

yes

### How was this patch tested?

new test

Closes #31930 from cloud-fan/view2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 14:34:51 +00:00
Liang-Chi Hsieh 115ed89a3c [SPARK-34366][SQL] Add interface for DS v2 metrics
### What changes were proposed in this pull request?

This patch proposes to add a few public API change to DS v2, to make DS v2 scan can report metrics to Spark.

Two public interfaces are added.

* `CustomMetric`: metric interface at the driver side. It basically defines how Spark aggregates task metrics with the same metric name.
* `CustomTaskMetric`: task metric reported at executors. It includes a name and long value. Spark will collect these metric values and update internal metrics.

There are two public methods added to existing public interfaces. They are optional to DS v2 implementations.

* `PartitionReader.currentMetricsValues()`: returns an array of CustomTaskMetric. Here is where the actual metrics values are collected. Empty array by default.
* `Scan.supportedCustomMetrics()`: returns an array of supported custom metrics `CustomMetric`. Empty array by default.

### Why are the changes needed?

In order to report custom metrics, we need some public API change in DS v2 to make it possible.

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

No

### How was this patch tested?

This only adds interfaces. In follow-up PRs where adding implementation there will be tests added. See #31451 and #31398 for some details and manual test there.

Closes #31476 from viirya/SPARK-34366.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 13:22:37 +00:00
Peter Toth 93a5d34f84 [SPARK-33482][SPARK-34756][SQL] Fix FileScan equality check
### What changes were proposed in this pull request?

This bug was introduced by SPARK-30428 at Apache Spark 3.0.0.
This PR fixes `FileScan.equals()`.

### Why are the changes needed?
- Without this fix `FileScan.equals` doesn't take `fileIndex` and `readSchema` into account.
- Partition filters and data filters added to `FileScan` (in #27112 and #27157) caused that canonicalized form of some `BatchScanExec` nodes don't match and this prevents some reuse possibilities.

### Does this PR introduce _any_ user-facing change?
Yes, before this fix incorrect reuse of `FileScan` and so `BatchScanExec` could have happed causing correctness issues.

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

Closes #31848 from peter-toth/SPARK-34756-fix-filescan-equality-check.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 17:01:16 +08:00
yi.wu e00afd31a7 [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself
### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

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

No.

### How was this patch tested?

Pass existing tests.

Closes #31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:38:43 +00:00
linzebing e768eaa908 [SPARK-34707][SQL] Code-gen broadcast nested loop join (left outer/right outer)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left outer (build right) and right outer (build left). Reference: `BroadcastNestedLoopJoinExec.codegenInner()` and `BroadcastNestedLoopJoinExec.outerJoin()`

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:
```scala
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left outer broadcast nested loop join", N) {
   val df = spark.range(N).selectExpr(s"id as k1").join(
     dim, col("k1") + 1 <= col("k2"), "left_outer")
   assert(df.queryExecution.sparkPlan.find(
     _.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined)
   df.noop()
}
```
Seeing 2x run time improvement:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left outer broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------------
left outer broadcast nested loop join wholestage off           3024           3698         953          6.9         144.2       1.0X
left outer broadcast nested loop join wholestage on            1512           1659         172         13.9          72.1       2.0X
```

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

No

### How was this patch tested?

Changed existing unit tests in `OuterJoinSuite` to cover codegen use cases.
Added unit test in WholeStageCodegenSuite.scala to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:
```scala
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_outer").explain("codegen")
```
Example generated code (`bnlj_doConsume_0` method):
```java
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, LeftOuter, ((k1#2L + 1) <= k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=16)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=16)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_foundMatch_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */       boolean bnlj_shouldOutputRow_0 = false;
/* 041 */
/* 042 */       boolean bnlj_isNull_2 = true;
/* 043 */       long bnlj_value_2 = -1L;
/* 044 */       if (bnlj_buildRow_0 != null) {
/* 045 */         long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 046 */         bnlj_isNull_2 = false;
/* 047 */         bnlj_value_2 = bnlj_value_1;
/* 048 */       }
/* 049 */
/* 050 */       long bnlj_value_4 = -1L;
/* 051 */
/* 052 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 053 */
/* 054 */       boolean bnlj_value_3 = false;
/* 055 */       bnlj_value_3 = bnlj_value_4 <= bnlj_value_2;
/* 056 */       if (!(false || !bnlj_value_3))
/* 057 */       {
/* 058 */         bnlj_shouldOutputRow_0 = true;
/* 059 */         bnlj_foundMatch_0 = true;
/* 060 */       }
/* 061 */       if (bnlj_arrayIndex_0 == bnlj_buildRowArray_0.length - 1 && !bnlj_foundMatch_0) {
/* 062 */         bnlj_buildRow_0 = null;
/* 063 */         bnlj_shouldOutputRow_0 = true;
/* 064 */       }
/* 065 */       if (bnlj_shouldOutputRow_0) {
/* 066 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 067 */
/* 068 */         boolean bnlj_isNull_9 = true;
/* 069 */         long bnlj_value_9 = -1L;
/* 070 */         if (bnlj_buildRow_0 != null) {
/* 071 */           long bnlj_value_8 = bnlj_buildRow_0.getLong(0);
/* 072 */           bnlj_isNull_9 = false;
/* 073 */           bnlj_value_9 = bnlj_value_8;
/* 074 */         }
/* 075 */         range_mutableStateArray_0[3].reset();
/* 076 */
/* 077 */         range_mutableStateArray_0[3].zeroOutNullBytes();
/* 078 */
/* 079 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 080 */
/* 081 */         if (bnlj_isNull_9) {
/* 082 */           range_mutableStateArray_0[3].setNullAt(1);
/* 083 */         } else {
/* 084 */           range_mutableStateArray_0[3].write(1, bnlj_value_9);
/* 085 */         }
/* 086 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 087 */
/* 088 */       }
/* 089 */     }
/* 090 */
/* 091 */   }
/* 092 */
/* 093 */   private void initRange(int idx) {
/* 094 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 095 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(16L);
/* 096 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 097 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 098 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 099 */     long partitionEnd;
/* 100 */
/* 101 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 102 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 103 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 104 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 105 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 106 */     } else {
/* 107 */       range_nextIndex_0 = st.longValue();
/* 108 */     }
/* 109 */     range_batchEnd_0 = range_nextIndex_0;
/* 110 */
/* 111 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 112 */     .multiply(step).add(start);
/* 113 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 114 */       partitionEnd = Long.MAX_VALUE;
/* 115 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 116 */       partitionEnd = Long.MIN_VALUE;
/* 117 */     } else {
/* 118 */       partitionEnd = end.longValue();
/* 119 */     }
/* 120 */
/* 121 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 122 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 123 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 124 */     if (range_numElementsTodo_0 < 0) {
/* 125 */       range_numElementsTodo_0 = 0;
/* 126 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 127 */       range_numElementsTodo_0++;
/* 128 */     }
/* 129 */   }
/* 130 */
/* 131 */   protected void processNext() throws java.io.IOException {
/* 132 */     // initialize Range
/* 133 */     if (!range_initRange_0) {
/* 134 */       range_initRange_0 = true;
/* 135 */       initRange(partitionIndex);
/* 136 */     }
/* 137 */
/* 138 */     while (true) {
/* 139 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 140 */         long range_nextBatchTodo_0;
/* 141 */         if (range_numElementsTodo_0 > 1000L) {
/* 142 */           range_nextBatchTodo_0 = 1000L;
/* 143 */           range_numElementsTodo_0 -= 1000L;
/* 144 */         } else {
/* 145 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 146 */           range_numElementsTodo_0 = 0;
/* 147 */           if (range_nextBatchTodo_0 == 0) break;
/* 148 */         }
/* 149 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 150 */       }
/* 151 */
/* 152 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 153 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 154 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 155 */
/* 156 */         // common sub-expressions
/* 157 */
/* 158 */         bnlj_doConsume_0(range_value_0);
/* 159 */
/* 160 */         if (shouldStop()) {
/* 161 */           range_nextIndex_0 = range_value_0 + 1L;
/* 162 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 163 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 164 */           return;
/* 165 */         }
/* 166 */
/* 167 */       }
/* 168 */       range_nextIndex_0 = range_batchEnd_0;
/* 169 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 170 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 171 */       range_taskContext_0.killTaskIfInterrupted();
/* 172 */     }
/* 173 */   }
/* 174 */
/* 175 */ }
```

Closes #31931 from linzebing/code-left-right-outer.

Authored-by: linzebing <linzebing1995@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:11:57 +00:00
hezuojiao 39542bb81f [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled
### What changes were proposed in this pull request?

This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption.

### Why are the changes needed?
Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message:
```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message=
org.apache.spark.shuffle.FetchFailedException: Stream is corrupted
	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845)
	at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
	at java.io.DataInputStream.readInt(DataInputStream.java:387)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:494)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.io.IOException: Stream is corrupted
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200)
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226)
	at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841)
	... 25 more

)
```

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

No

### How was this patch tested?

New tests.

Closes #31898 from hezuojiao/fetch_shuffle_in_batch.

Authored-by: hezuojiao <hezuojiao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 13:06:12 -07:00
tanel.kiis@gmail.com 51cf0cadea [SPARK-34812][SQL] RowNumberLike and RankLike should not be nullable
### What changes were proposed in this pull request?

Marked `RowNumberLike` and `RankLike` as not-nullable.

### Why are the changes needed?

`RowNumberLike` and `RankLike` SQL expressions never return null value. Marking them as non-nullable can have some performance benefits, because some optimizer rules apply only to non-nullable expressions

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

No

### How was this patch tested?

Did not find any existing tests on the nullability of aggregate functions.
Plan stability suite partially covers this.

Closes #31924 from tanelk/SPARK-34812_nullability.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 14:55:43 +00:00
woyumen4597 f44608a8c0 [SPARK-34800][SQL] Use fine-grained lock in SessionCatalog.tableExists
### What changes were proposed in this pull request?
Use fine-grained lock in SessionCatalog.tableExists, in order to lock currentDB variable rather than lock `tableExists` method which will block inner external catalog's behaviour.

### Why are the changes needed?
We have modified the underlying hive meta store which a different hive  database is placed in its own shard for performance. However, we found that the synchronized lock  limits the concurrency.

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

Closes #31891 from woyumen4597/SPARK-34800.

Authored-by: woyumen4597 <woyumen4597@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 09:03:46 +00:00
Terry Kim 7953fcdb56 [SPARK-34700][SQL] SessionCatalog's temporary view related APIs should take/return more concrete types
### What changes were proposed in this pull request?

Now that all the temporary views are wrapped with `TemporaryViewRelation`(#31273, #31652, and #31825), this PR proposes to update `SessionCatalog`'s APIs for temporary views to take or return more concrete types.

APIs that will take `TemporaryViewRelation` instead of `LogicalPlan`:
```
createTempView, createGlobalTempView, alterTempViewDefinition
```

APIs that will return `TemporaryViewRelation` instead of `LogicalPlan`:
```
getRawTempView, getRawGlobalTempView
```

APIs that will return `View` instead of `LogicalPlan`:
```
getTempView, getGlobalTempView, lookupTempView
```

### Why are the changes needed?

Internal refactoring to work with more concrete types.

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

No, this is internal refactoring.

### How was this patch tested?

Updated existing tests affected by the refactoring.

Closes #31906 from imback82/use_temporary_view_relation.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:17:54 +00:00
yi.wu e4bb97526c [SPARK-34089][CORE] HybridRowQueue should respect the configured memory mode
### What changes were proposed in this pull request?

This PR fixes the `HybridRowQueue ` to respect the configured memory mode.

Besides, this PR also refactored the constructor of `MemoryConsumer` to accept the memory mode explicitly.

### Why are the changes needed?

`HybridRowQueue` supports both onHeap and offHeap manipulation. But it inherited the wrong `MemoryConsumer` constructor, which hard-coded the memory mode to `onHeap`.

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

No. (Maybe yes in some cases where users can't complete the job before could complete successfully after the fix because of `HybridRowQueue` is able to spill under offHeap mode now. )

### How was this patch tested?

Updated the existing test to make it test both offHeap and onHeap modes.

Closes #31152 from Ngone51/fix-MemoryConsumer-memorymode.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:12:08 +00:00
HyukjinKwon ec70467d4d [SPARK-34815][SQL] Update CSVBenchmark
### What changes were proposed in this pull request?

This PR updates CSVBenchmark especially we have a fix like https://github.com/apache/spark/pull/31858 that could potentially improve the performance.

### Why are the changes needed?

To have the updated benchmark results.

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

No.

### How was this patch tested?

Manually ran the benchmark

Closes #31917 from HyukjinKwon/SPARK-34815.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 10:49:53 +03:00
Jungtaek Lim (HeartSaVioR) 121883b1a5 [SPARK-34383][SS] Optimize WAL commit phase via reducing cost of filesystem operations
### What changes were proposed in this pull request?

This PR proposes to optimize WAL commit phase via following changes:

* cache offset log to avoid FS get operation per batch
* just directly delete instead of employing FS list operation on purge

### Why are the changes needed?

There're inefficiency on WAL commit phase which can be easily optimized via using a small driver memory.

1. To provide the offset metadata to source side (via `source.commit()`), we read offset metadata for previous batch from file system, which is probably written by this driver in previous batches. Caching it into driver memory would reduce the get operation.
2. Spark calls purge against offset log & commit log per batch, which calls list operation. If the previous batch succeeded to purge, the current batch just needs to check one batch which can be simply done via direct delete operation, instead of calling list operation.

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

No.

### How was this patch tested?

Manually tested with additional debug log. (Verified that cache is used, cache keeps the size as 2, only one delete call is used instead of list call)

Did some experiment with simple rate to console query. (NOTE: wasn't done with master branch - tested against Spark 2.4.x, but WAL commit phase hasn't been changed AFAIK during these versions)

AWS S3 + S3 guard:

> before the patch

<img width="1075" alt="aws-before" src="https://user-images.githubusercontent.com/1317309/107108721-6cc54380-687d-11eb-8f10-b906b9d58397.png">

> after the patch

<img width="1071" alt="aws-after" src="https://user-images.githubusercontent.com/1317309/107108724-7189f780-687d-11eb-88da-26912ac15c85.png">

Azure:

> before the patch

<img width="1074" alt="azure-before" src="https://user-images.githubusercontent.com/1317309/107108726-75b61500-687d-11eb-8c06-9048fa10ff9a.png">

> after the patch

<img width="1069" alt="azure-after" src="https://user-images.githubusercontent.com/1317309/107108729-79e23280-687d-11eb-8d97-e7f3aeec51be.png">

Closes #31495 from HeartSaVioR/SPARK-34383.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
2021-03-22 08:47:07 +01:00
Cheng Su f8838fe82b [SPARK-34708][SQL] Code-gen for left semi/anti broadcast nested loop join (build right side)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left semi / left anti BroadcastNestedLoopJoin (build side is right side). The execution code path for build left side cannot fit into whole stage code-gen framework, so only add the code-gen for build right side here.

Reference: the iterator (non-code-gen) code path is `BroadcastNestedLoopJoinExec.leftExistenceJoin()` with `BuildRight`.

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:

```
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left semi broadcast nested loop join", N) {
  park.range(N).selectExpr(s"id as k1").join(
    dim, col("k1") + 1 <= col("k2"), "left_semi")
}
```

Seeing 5x run time improvement:

```
Running benchmark: left semi broadcast nested loop join
  Running case: left semi broadcast nested loop join codegen off
  Stopped after 2 iterations, 6958 ms
  Running case: left semi broadcast nested loop join codegen on
  Stopped after 5 iterations, 3383 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi broadcast nested loop join:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------------------------------------------------------
left semi broadcast nested loop join codegen off           3434           3479          65          6.1         163.7       1.0X
left semi broadcast nested loop join codegen on             672            677           5         31.2          32.1       5.1X
```

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

No.

### How was this patch tested?

Changed existing unit test in `ExistenceJoinSuite.scala` to cover all code paths:
* left semi/anti + empty right side + empty condition
* left semi/anti + non-empty right side + empty condition
* left semi/anti + right side + non-empty condition

Added unit test in `WholeStageCodegenSuite.scala` to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_semi").explain("codegen")
```

Example generated code (`bnlj_doConsume_0` method):
This is for left semi join. The generated code for left anti join is mostly to be same as here, except L55 to be `if (bnlj_findMatchedRow_0 == false) {`.
```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) Project [id#0L AS k1#2L]
+- *(2) BroadcastNestedLoopJoin BuildRight, LeftSemi, ((id#0L + 1) <= k2#6L)
   :- *(2) Range (0, 4, step=1, splits=2)
   +- BroadcastExchange IdentityBroadcastMode, [id=#23]
      +- *(1) Project [id#4L AS k2#6L]
         +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 031 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_findMatchedRow_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */
/* 041 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 042 */
/* 043 */       long bnlj_value_3 = -1L;
/* 044 */
/* 045 */       bnlj_value_3 = bnlj_expr_0_0 + 1L;
/* 046 */
/* 047 */       boolean bnlj_value_2 = false;
/* 048 */       bnlj_value_2 = bnlj_value_3 <= bnlj_value_1;
/* 049 */       if (!(false || !bnlj_value_2))
/* 050 */       {
/* 051 */         bnlj_findMatchedRow_0 = true;
/* 052 */         break;
/* 053 */       }
/* 054 */     }
/* 055 */     if (bnlj_findMatchedRow_0 == true) {
/* 056 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 057 */
/* 058 */       // common sub-expressions
/* 059 */
/* 060 */       range_mutableStateArray_0[3].reset();
/* 061 */
/* 062 */       range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 063 */       append((range_mutableStateArray_0[3].getRow()).copy());
/* 064 */
/* 065 */     }
/* 066 */
/* 067 */   }
/* 068 */
/* 069 */   private void initRange(int idx) {
/* 070 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 071 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 072 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 073 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 074 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 075 */     long partitionEnd;
/* 076 */
/* 077 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 078 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 079 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 080 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 081 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 082 */     } else {
/* 083 */       range_nextIndex_0 = st.longValue();
/* 084 */     }
/* 085 */     range_batchEnd_0 = range_nextIndex_0;
/* 086 */
/* 087 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 088 */     .multiply(step).add(start);
/* 089 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 090 */       partitionEnd = Long.MAX_VALUE;
/* 091 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 092 */       partitionEnd = Long.MIN_VALUE;
/* 093 */     } else {
/* 094 */       partitionEnd = end.longValue();
/* 095 */     }
/* 096 */
/* 097 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 098 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 099 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 100 */     if (range_numElementsTodo_0 < 0) {
/* 101 */       range_numElementsTodo_0 = 0;
/* 102 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 103 */       range_numElementsTodo_0++;
/* 104 */     }
/* 105 */   }
/* 106 */
/* 107 */   protected void processNext() throws java.io.IOException {
/* 108 */     // initialize Range
/* 109 */     if (!range_initRange_0) {
/* 110 */       range_initRange_0 = true;
/* 111 */       initRange(partitionIndex);
/* 112 */     }
/* 113 */
/* 114 */     while (true) {
/* 115 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 116 */         long range_nextBatchTodo_0;
/* 117 */         if (range_numElementsTodo_0 > 1000L) {
/* 118 */           range_nextBatchTodo_0 = 1000L;
/* 119 */           range_numElementsTodo_0 -= 1000L;
/* 120 */         } else {
/* 121 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 122 */           range_numElementsTodo_0 = 0;
/* 123 */           if (range_nextBatchTodo_0 == 0) break;
/* 124 */         }
/* 125 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 126 */       }
/* 127 */
/* 128 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 129 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 130 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 131 */
/* 132 */         bnlj_doConsume_0(range_value_0);
/* 133 */
/* 134 */         if (shouldStop()) {
/* 135 */           range_nextIndex_0 = range_value_0 + 1L;
/* 136 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 137 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 138 */           return;
/* 139 */         }
/* 140 */
/* 141 */       }
/* 142 */       range_nextIndex_0 = range_batchEnd_0;
/* 143 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 144 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 145 */       range_taskContext_0.killTaskIfInterrupted();
/* 146 */     }
/* 147 */   }
/* 148 */
/* 149 */ }
```

Closes #31874 from c21/code-semi-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 07:31:16 +00:00
Yuanjian Li 45235ac4bc [SPARK-34748][SS] Create a rule of the analysis logic for streaming write
### What changes were proposed in this pull request?
- Create a new rule `ResolveStreamWrite` for all analysis logic for streaming write.
- Add corresponding logical plans `WriteToStreamStatement` and `WriteToStream`.

### Why are the changes needed?
Currently, the analysis logic for streaming write is mixed in StreamingQueryManager. If we create a specific analyzer rule and separated logical plans, it should be helpful for further extension.

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

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

Closes #31842 from xuanyuanking/SPARK-34748.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 06:39:39 +00:00
Josh Soref f4de93efb0 [MINOR][SQL] Spelling: filters - PushedFilers
### What changes were proposed in this pull request?
Consistently correct the spelling of `PushedFilters`

### Why are the changes needed?
bersprockets noted that it's wrong

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

Technically, I think it does. Practically, neither Google nor GitHub show anyone using `pushedFilers` outside of forks (or the discussion about fixing it started at https://github.com/apache/spark/pull/30323#issuecomment-725568719)

### How was this patch tested?
None beyond CI in the previous PR

Closes #30678 from jsoref/spelling-filters.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 08:00:12 +03:00
Dongjoon Hyun c5fd94f119 [SPARK-34772][TESTS][FOLLOWUP] Disable a test case using Hive 1.2.1 in Java9+ environment
### What changes were proposed in this pull request?

This PR aims to disable a new test case using Hive 1.2.1 from Java9+ test environment.

### Why are the changes needed?

[HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113) upgraded Datanucleus to 4.x at Hive 2.0. Datanucleus 3.x doesn't support Java9+.

**Java 9+ Environment**
```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] *** 1 TEST FAILED ***
[error] Failed: Total 1, Failed 1, Errors 0, Passed 0
[error] Failed tests:
[error] 	org.apache.spark.sql.hive.HiveSparkSubmitSuite
[error] (hive / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 328 s (05:28), completed Mar 21, 2021, 5:32:39 PM
```

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

Fix the UT in Java9+ environment.

### How was this patch tested?

Manually.

```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] HiveSparkSubmitSuite:
[info] - SPARK-34772: RebaseDateTime loadRebaseRecords should use Spark classloader instead of context !!! CANCELED !!! (26 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:344)
```

Closes #31916 from dongjoon-hyun/SPARK-HiveSparkSubmitSuite.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 17:59:55 -07:00
Kousuke Saruta 94fd6cb0ce [SPARK-34636][FOLLOWUP][SQL] Fix an incompatible behavior of UnresolvedAttribute.sql
### What changes were proposed in this pull request?

This PR fixes an incompatible behavior introduced by #31754.
The problem is that quoted name parts represented as a string are given to the constructor of `UnresolvedAttribute` which takes single string parameter, `sql` method invocation against the `UnresolvedAttrribute` returns different result than before.

One example is ``` UnresolvedAttribute("`a.b`").sql ```. This  returned `a.b` before but it doesn't now.

See [this duscussion](https://github.com/apache/spark/pull/31754/files#r597181927) for more details.

### Why are the changes needed?

For compatibility.

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

No.

### How was this patch tested?

New assertion.

Closes #31885 from sarutak/followup-SPARK-34636.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-20 14:44:36 -07:00
Yuming Wang 908318f30d [SPARK-28220][SQL] Improve PropagateEmptyRelation to support join with false condition
### What changes were proposed in this pull request?

Improve `PropagateEmptyRelation` to support join with false condition. For example:
```sql
SELECT * FROM t1 LEFT JOIN t2 ON false
```

Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- BroadcastNestedLoopJoin BuildRight, LeftOuter, false
   :- FileScan parquet default.t1[a#4L]
   +- BroadcastExchange IdentityBroadcastMode, [id=#40]
      +- FileScan parquet default.t2[b#5L]
```

After this pr:
```
== Physical Plan ==
*(1) Project [a#4L, null AS b#5L]
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[a#4L]
```

### Why are the changes needed?

Avoid `BroadcastNestedLoopJoin` to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31857 from wangyum/SPARK-28220.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-20 22:57:02 +08:00
Kent Yao 2cdedef2a0 [SPARK-34128][SQL] Suppress undesirable TTransportException warnings involved in THRIFT-4805
### What changes were proposed in this pull request?

Since Spark 3.0, the `libthrift` has been bumped up from 0.9.3 to 0.12.0.

Due to THRIFT-4805, The SparkThrift Server will print annoying TExceptions. For example, the current thrift server module test in Github action workflow outputs more than 200MB of data for this error only
```java
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
```

I checked the latest `hive-service-rpc` module in the maven center,  https://mvnrepository.com/artifact/org.apache.hive/hive-service-rpc/3.1.2.  It still uses the 0.9.3 version.

Unfortunately, I tried the newly released `libthrift 0.14.1`(w/o shading it), it breaks the metastore client side.

```scala
java.lang.NoSuchMethodError: org.apache.thrift.transport.TSocket.<init>(Ljava/lang/String;II)V
```
On the Thrift side, they just muted it see https://issues.apache.org/jira/browse/THRIFT-4805

So in this PR, I add a filter to suppress the warning

### Why are the changes needed?

if the log is too large, the Github action might truncate it. We need to reduce useless output.

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

no

### How was this patch tested?

```build/sbt "hive-thriftserver/testOnly *ThriftServerQueryTestSuite" -Phive-thriftserver``` locally

#### before

```java
[info] - count.sql (1 second, 537 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
14:09:53.233 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - group-analytics.sql (4 seconds, 282 milliseconds)

[info] - csv-functions.sql (400 milliseconds)
14:09:24.234 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - datetime-formatting-invalid.sql (349 milliseconds)
14:09:26.544 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - except.sql (2 seconds, 309 milliseconds)
14:09:27.782 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)
[info] - string-functions.sql (1 second, 237 milliseconds)
14:09:27.835 WARN org.apache.spark.sql.execution.datasources.DataSource: All paths were ignored:

14:09:29.266 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	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)

```

#### after

```java

[info] - null-propagation.sql (181 milliseconds)
[info] - operators.sql (1 second, 772 milliseconds)
[info] - change-column.sql (241 milliseconds)
[info] - count.sql (1 second, 665 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
[info] - group-analytics.sql (3 seconds, 926 milliseconds)
[info] - inline-table.sql (247 milliseconds)
[info] - comparator.sql (223 milliseconds)
[info] - show-tblproperties.sql (148 milliseconds)
[info] - timezone.sql (105 milliseconds)
[info] - parse-schema-string.sql (193 milliseconds)
```

Closes #31895 from yaooqinn/SPARK-34128-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 21:15:28 -07:00
Cheng Su 2ff0032e01 [SPARK-34796][SQL] Initialize counter variable for LIMIT code-gen in doProduce()
### What changes were proposed in this pull request?

This PR is to fix the LIMIT code-gen bug in https://issues.apache.org/jira/browse/SPARK-34796, where the counter variable from `BaseLimitExec` is not initialized but used in code-gen. This is because the limit counter variable will be used in upstream operators (LIMIT's child plan, e.g. `ColumnarToRowExec` operator for early termination), but in the same stage, there can be some operators doing the shortcut and not calling `BaseLimitExec`'s `doConsume()`, e.g. [HashJoin.codegenInner](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L402). So if we have query that `LocalLimit - BroadcastHashJoin - FileScan` in the same stage, the whole stage code-gen compilation will be failed.

Here is an example:

```
  test("failed limit query") {
    withTable("left_table", "empty_right_table", "output_table") {
      spark.range(5).toDF("k").write.saveAsTable("left_table")
      spark.range(0).toDF("k").write.saveAsTable("empty_right_table")

      withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
        spark.sql("CREATE TABLE output_table (k INT) USING parquet")
        spark.sql(
          s"""
             |INSERT INTO TABLE output_table
             |SELECT t1.k FROM left_table t1
             |JOIN empty_right_table t2
             |ON t1.k = t2.k
             |LIMIT 3
             |""".stripMargin)
      }
    }
  }
```

Query plan:

```
Execute InsertIntoHadoopFsRelationCommand file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table, false, Parquet, Map(path -> file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table), Append, CatalogTable(
Database: default
Table: output_table
Created Time: Thu Mar 18 21:46:26 PDT 2021
Last Access: UNKNOWN
Created By: Spark 3.2.0-SNAPSHOT
Type: MANAGED
Provider: parquet
Location: file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table
Schema: root
 |-- k: integer (nullable = true)
), org.apache.spark.sql.execution.datasources.InMemoryFileIndexb25d08b, [k]
+- *(3) Project [ansi_cast(k#228L as int) AS k#231]
   +- *(3) GlobalLimit 3
      +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#179]
         +- *(2) LocalLimit 3
            +- *(2) Project [k#228L]
               +- *(2) BroadcastHashJoin [k#228L], [k#229L], Inner, BuildRight, false
                  :- *(2) Filter isnotnull(k#228L)
                  :  +- *(2) ColumnarToRow
                  :     +- FileScan parquet default.left_table[k#228L] Batched: true, DataFilters: [isnotnull(k#228L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
                  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#173]
                     +- *(1) Filter isnotnull(k#229L)
                        +- *(1) ColumnarToRow
                           +- FileScan parquet default.empty_right_table[k#229L] Batched: true, DataFilters: [isnotnull(k#229L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
```

Codegen failure - https://gist.github.com/c21/ea760c75b546d903247582be656d9d66 .

The uninitialized variable `_limit_counter_1` from `LocalLimitExec` is referenced in `ColumnarToRowExec`, but `BroadcastHashJoinExec` does not call `LocalLimitExec.doConsume()` to initialize the counter variable.

The fix is to move the counter variable initialization to `doProduce()`, as in whole stage code-gen framework, `doProduce()` will definitely be called if upstream operators `doProduce()`/`doConsume()` is called.

Note: this only happens in AQE disabled case, because we have an AQE optimization rule [EliminateUnnecessaryJoin](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala#L69) to change the whole query to an empty `LocalRelation` if inner join broadcast side is empty with AQE enabled.

### Why are the changes needed?

Fix query failure.

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

No.

### How was this patch tested?

Added unit test in `SQLQuerySuite.scala`.

Closes #31892 from c21/limit-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:20:52 +09:00
tanel.kiis@gmail.com 620cae098c [SPARK-33122][SQL] Remove redundant aggregates in the Optimzier
### What changes were proposed in this pull request?

Added optimizer rule `RemoveRedundantAggregates`. It removes redundant aggregates from a query plan. A redundant aggregate is an aggregate whose only goal is to keep distinct values, while its parent aggregate would ignore duplicate values.

The affected part of the query plan for TPCDS q87:

Before:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#785]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
            +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                  +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                     +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#724]
                        +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                           +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                              :- ...
```

After:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#751]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#694]
            +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                  :- ...
```

### Why are the changes needed?

Performance improvements - few TPCDS queries have these kinds of duplicate aggregates.

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

No

### How was this patch tested?

UT

Benchmarks (sf=5):

OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Linux 5.8.13-arch1-1
Intel(R) Core(TM) i5-6500 CPU  3.20GHz

| Query | Before  | After | Speedup |
| ------| ------- | ------| ------- |
| q14a | 44s | 44s | 1x |
| q14b | 41s | 41s | 1x |
| q38  | 6.5s | 5.9s | 1.1x |
| q87  | 7.2s | 6.8s | 1.1x |
| q14a-v2.7 | 55s | 53s | 1x |

Closes #30018 from tanelk/SPARK-33122.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:16:39 +09:00
Liang-Chi Hsieh 7a8a600995 [SPARK-34776][SQL] Nested column pruning should not prune Window produced attributes
### What changes were proposed in this pull request?

This patch proposes to fix a bug related to `NestedColumnAliasing`. The root cause is `Window`  doesn't override `producedAttributes` so `NestedColumnAliasing` rule wrongly prune attributes produced by `Window`.

The master and branch-3.1 both have this issue.

### Why are the changes needed?

It is needed to fix a bug of nested column pruning.

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

No

### How was this patch tested?

Unit test.

Closes #31897 from viirya/SPARK-34776.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 11:44:02 -07:00
Max Gekk 089c3b77e1 [SPARK-34793][SQL] Prohibit saving of day-time and year-month intervals
### What changes were proposed in this pull request?
For all built-in datasources, prohibit saving of year-month and day-time intervals that were introduced by SPARK-27793. We plan to support saving of such types at the milestone 2, see SPARK-27790.

### Why are the changes needed?
To improve user experience with Spark SQL, and print nicer error message. Current error message might confuse users:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
21/03/18 22:44:35 ERROR FileFormatWriter: Aborting job 8de402d7-ab69-4dc0-aa8e-14ef06bd2d6b.
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.66 executor driver): org.apache.spark.SparkException: Task failed while writing rows.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.taskFailedWhileWritingRowsError(QueryExecutionErrors.scala:418)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:298)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:211)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	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)
Caused by: java.lang.RuntimeException: Failed to convert value 1 (class of class java.lang.Integer}) with the type of YearMonthIntervalType to JSON.
	at scala.sys.package$.error(package.scala:30)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23(JacksonGenerator.scala:179)
	at org.apache.spark.sql.catalyst.json.JacksonGenerator.$anonfun$makeWriter$23$adapted(JacksonGenerator.scala:176)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above:
```
scala> Seq(java.time.Period.ofMonths(1)).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```

### How was this patch tested?
1. Checked nested intervals:
```
scala> spark.range(1).selectExpr("""struct(timestamp'2021-01-02 00:01:02' - timestamp'2021-01-01 00:00:00')""").write.mode("overwrite").parquet("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
scala> Seq(Seq(java.time.Period.ofMonths(1))).toDF.write.mode("overwrite").json("/Users/maximgekk/tmp/123")
org.apache.spark.sql.AnalysisException: Cannot save interval data type into external storage.
```
2. By running existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2DataFrameSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```

Closes #31884 from MaxGekk/ban-save-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-19 18:47:53 +03:00
Hongyi Zhang 6f89cdfb0c [SPARK-34798][SQL][TESTS] Fix incorrect join condition
### What changes were proposed in this pull request?

join condition 'a.attr == 'c.attr check the reference of  these 2 objects which will always returns false. we need to use === instead

### Why are the changes needed?

Although this join condition always false doesn't break the test but it is not what we expected. We should fix it to avoid future confusing

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

No

### How was this patch tested?

UT

Closes #31890 from opensky142857/SPARK-34798.

Authored-by: Hongyi Zhang <hongyzhang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-19 23:35:15 +08:00
Wenchen Fan 4b4f8e2a25 [SPARK-34558][SQL][FOLLOWUP] Use final Hadoop conf to instantiate FileSystem in SharedState
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31671

https://github.com/apache/spark/pull/31671 has an unexpected behavior change that it uses a different Hadoop conf (`sparkContext.hadoopConfiguration`) to instantiate `FileSystem`, which is used to qualify the warehouse path. Before https://github.com/apache/spark/pull/31671 , the Hadoop conf to instantiate `FileSystem` is `session.sessionState.newHadoopConf()`.

More specifically, `session.sessionState.newHadoopConf()` has more conf entries:
1. it includes configs from `SharedState.initialConfigs`
2. in includes configs from `sparkContext.conf`

This PR updates `SharedState` to use the final Hadoop conf to instantiate `FileSystem`.

### Why are the changes needed?

fix behavior change

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

yes, the behavior will be the same before https://github.com/apache/spark/pull/31671

### How was this patch tested?

manually check the log of `FileSystem` and verify the passed in configs.

Closes #31868 from cloud-fan/followup.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-19 22:02:15 +08:00
ulysses-you 58509565f8 [SPARK-34772][SQL] RebaseDateTime loadRebaseRecords should use Spark classloader instead of context
### What changes were proposed in this pull request?

Change context classloader to Spark classloader at `RebaseDateTime.loadRebaseRecords`

### Why are the changes needed?

With custom `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars`.

Spark would use date formatter in `HiveShim` that convert `date` to `string`, if we set `spark.sql.legacy.timeParserPolicy=LEGACY` and the partition type is `date` the `RebaseDateTime` code will be invoked. At that moment, if `RebaseDateTime` is initialized the first time then context class loader is `IsolatedClientLoader`. Such error msg would throw:

```
java.lang.IllegalArgumentException: argument "src" is null
  at com.fasterxml.jackson.databind.ObjectMapper._assertNotNull(ObjectMapper.java:4413)
  at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3157)
  at com.fasterxml.jackson.module.scala.ScalaObjectMapper.readValue(ScalaObjectMapper.scala:187)
  at com.fasterxml.jackson.module.scala.ScalaObjectMapper.readValue$(ScalaObjectMapper.scala:186)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$$anon$1.readValue(RebaseDateTime.scala:267)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.loadRebaseRecords(RebaseDateTime.scala:269)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.<init>(RebaseDateTime.scala:291)
  at org.apache.spark.sql.catalyst.util.RebaseDateTime$.<clinit>(RebaseDateTime.scala)
  at org.apache.spark.sql.catalyst.util.DateTimeUtils$.toJavaDate(DateTimeUtils.scala:109)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format(DateFormatter.scala:95)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format$(DateFormatter.scala:94)
  at org.apache.spark.sql.catalyst.util.LegacySimpleDateFormatter.format(DateFormatter.scala:138)
  at org.apache.spark.sql.hive.client.Shim_v0_13$ExtractableLiteral$1$.unapply(HiveShim.scala:661)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:785)
  at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826)
```

```
java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.sql.catalyst.util.RebaseDateTime$
  at org.apache.spark.sql.catalyst.util.DateTimeUtils$.toJavaDate(DateTimeUtils.scala:109)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format(DateFormatter.scala:95)
  at org.apache.spark.sql.catalyst.util.LegacyDateFormatter.format$(DateFormatter.scala:94)
  at org.apache.spark.sql.catalyst.util.LegacySimpleDateFormatter.format(DateFormatter.scala:138)
  at org.apache.spark.sql.hive.client.Shim_v0_13$ExtractableLiteral$1$.unapply(HiveShim.scala:661)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:785)
  at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826)
  at scala.collection.immutable.Stream.flatMap(Stream.scala:493)
  at org.apache.spark.sql.hive.client.Shim_v0_13.convertFilters(HiveShim.scala:826)
  at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:848)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:749)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:291)
  at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:224)
  at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:223)
  at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:273)
  at org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:747)
  at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$listPartitionsByFilter$1(HiveExternalCatalog.scala:1273)
```

The reproduce steps:
1. `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars`.
2. `CREATE TABLE t (c int) PARTITIONED BY (p date)`
3. `SET spark.sql.legacy.timeParserPolicy=LEGACY`
4. `SELECT * FROM t WHERE p='2021-01-01'`

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

Yes, bug fix.

### How was this patch tested?

pass `org.apache.spark.sql.catalyst.util.RebaseDateTimeSuite` and add new unit test to `HiveSparkSubmitSuite.scala`.

Closes #31864 from ulysses-you/SPARK-34772.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-19 12:51:43 +08:00
Max Gekk a48b2086dd [SPARK-34761][SQL] Support add/subtract of a day-time interval to/from a timestamp
### What changes were proposed in this pull request?
Support `timestamp +/- day-time interval`. In the PR, I propose to extend the `TimeAdd` expression and support `DayTimeIntervalType` as the `interval` parameter. The expression invokes the new method `DateTimeUtils.timestampAddDayTime()` which splits the input day-time interval to `days` and `microsecond adjustment` of a day, and adds `days` (and the microseconds) to a local timestamp derived from the given timestamp at the given time zone.  The resulted local timestamp is converted back to the offset in microseconds since the epoch.

Also I updated the rules that handle `CalendarIntervalType` and produce `TimeAdd` to take into account new type `DateTimeIntervalType` for the `interval` parameter of `TimeAdd`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over timestamps and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31855 from MaxGekk/timestamp-add-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-19 04:02:34 +00:00
Karuppayya Rajendran 0a58029d52 [SPARK-31897][SQL] Enable codegen for GenerateExec
### What changes were proposed in this pull request?
Enabling codegen for GenerateExec

### Why are the changes needed?
To leverage code generation for Generators

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

### How was this patch tested?
- UT tests added

### Benchmark
```
case class Data(value1: Float, value2: Map[String, String], value3: String)
val path = "<path>"

val numRecords = Seq(10000000, 100000000)
numRecords.map {
  recordCount =>
    import java.util.concurrent.TimeUnit.NANOSECONDS

    val srcDF = spark.range(recordCount).map {
      x => Data(x.toFloat, Map(x.toString -> x.toString ), s"value3$x")
    }.select($"value1", explode($"value2"), $"value3")
    val start = System.nanoTime()
    srcDF
      .write
      .mode("overwrite")
      .parquet(s"$path/$recordCount")
    val end = System.nanoTime()
    val diff = end - start
    (recordCount, NANOSECONDS.toMillis(diff))
}
```
**With codegen**:
```
res0: Seq[(Int, Long)] = List((10000000,13989), (100000000,129625))
```
**Without codegen**:
```
res0: Seq[(Int, Long)] = List((10000000,15736), (100000000,150399))
```

Closes #28715 from karuppayya/SPARK-31897.

Lead-authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com>
Co-authored-by: Karuppayya Rajendran <karuppayya.rajendran@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-18 20:50:28 -07:00
Kousuke Saruta 07ee73234f [SPARK-34747][SQL][DOCS] Add virtual operators to the built-in function document
### What changes were proposed in this pull request?

This PR fix an issue that virtual operators (`||`, `!=`, `<>`, `between` and `case`) are absent from the Spark SQL Built-in functions document.

### Why are the changes needed?

The document should explain about all the supported built-in operators.

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

No.

### How was this patch tested?

Built the document with `SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_PYTHONDOC=1 bundler exec jekyll build` and then, confirmed the document.

![neq1](https://user-images.githubusercontent.com/4736016/111192859-e2e76380-85fc-11eb-89c9-75916a5e856a.png)
![neq2](https://user-images.githubusercontent.com/4736016/111192874-e7ac1780-85fc-11eb-9a9b-c504265b373f.png)
![between](https://user-images.githubusercontent.com/4736016/111192898-eda1f880-85fc-11eb-992d-cf80c544ec27.png)
![case](https://user-images.githubusercontent.com/4736016/111192918-f266ac80-85fc-11eb-9306-5dbc413a0cdb.png)
![double_pipe](https://user-images.githubusercontent.com/4736016/111192952-fb577e00-85fc-11eb-932e-385e5c2a5205.png)

Closes #31841 from sarutak/builtin-op-doc.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-19 10:19:26 +09:00
Cheng Su 8207e2f65c [SPARK-34781][SQL] Eliminate LEFT SEMI/ANTI joins to its left child side in AQE
### What changes were proposed in this pull request?

In `EliminateJoinToEmptyRelation.scala`, we can extend it to cover more cases for LEFT SEMI and LEFT ANI joins:

* Join is left semi join, join right side is non-empty and condition is empty. Eliminate join to its left side.
* Join is left anti join, join right side is empty. Eliminate join to its left side.

Given we eliminate join to its left side here, renaming the current optimization rule to `EliminateUnnecessaryJoin` instead.
In addition, also change to use `checkRowCount()` to check run time row count, instead of using `EmptyHashedRelation`. So this can cover `BroadcastNestedLoopJoin` as well. (`BroadcastNestedLoopJoin`'s broadcast side is `Array[InternalRow]`, not `HashedRelation`).

### Why are the changes needed?

Cover more join cases, and improve query performance for affected queries.

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

No.

### How was this patch tested?

Added unit tests in `AdaptiveQueryExecSuite.scala`.

Closes #31873 from c21/aqe-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-19 09:41:52 +09:00
yi.wu d99135b66a [SPARK-34741][SQL] MergeIntoTable should avoid ambiguous reference in UpdateAction
### What changes were proposed in this pull request?

This PR proposes to deduplicate the source table when there're conflicting attributes between the target table and the source table.

### Why are the changes needed?

When resolving the `UpdateAction`, which could reference attributes from both target and source tables,  Spark should know clearly where the attribute comes from when there're conflicting attributes instead of picking up a random one.

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

No.

### How was this patch tested?

Added a unit test and updated existing tests.

Closes #31835 from Ngone51/dedup-MergeIntoTable.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-18 15:54:41 +08:00
Luan 25e7d1ceee [SPARK-34728][SQL] Remove all SQLConf.get if extends from SQLConfHelper
### What changes were proposed in this pull request?

Remove all SQLConf.get to conf if extends from SQLConfHelper

### Why are the changes needed?

Clean up code.

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

No

### How was this patch tested?

Existing unit tests.

Closes #31822 from leoluan2009/SPARK-34728.

Authored-by: Luan <luanxuedong2009@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 15:04:41 +09:00
yi.wu 4d90c5dc0e [SPARK-34087][SQL] Fix memory leak of ExecutionListenerBus
### What changes were proposed in this pull request?

This PR proposes an alternative way to fix the memory leak of `ExecutionListenerBus`, which would automatically clean them up.

Basically, the idea is to add `registerSparkListenerForCleanup` to `ContextCleaner`, so we can remove the `ExecutionListenerBus` from `LiveListenerBus` when the `SparkSession` is GC'ed.

On the other hand, to make the `SparkSession` GC-able, we need to get rid of the reference of `SparkSession` in `ExecutionListenerBus`. Therefore, we introduced the `sessionUUID`, which is a unique identifier for SparkSession, to replace the  `SparkSession` object.

Note that, the proposal wouldn't take effect when `spark.cleaner.referenceTracking=false` since it depends on `ContextCleaner`.

### Why are the changes needed?

Fix the memory leak caused by `ExecutionListenerBus` mentioned in SPARK-34087.

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

Yes, save memory for users.

### How was this patch tested?

Added unit test.

Closes #31839 from Ngone51/fix-mem-leak-of-ExecutionListenerBus.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 13:27:03 +09:00
Kousuke Saruta c5cadfefdf [SPARK-34762][BUILD] Fix the build failure with Scala 2.13 which is related to commons-cli
### What changes were proposed in this pull request?

This PR fixes the build failure with Scala 2.13 which is related to `commons-cli`.
The last few days, build with Scala 2.13 on GA continues to fail and the error message says like as follows.
```
[error] /home/runner/work/spark/spark/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java:26:1:  error: package org.apache.commons.cli does not exist
1278[error] import org.apache.commons.cli.GnuParser;
```
The reason is that `mvn help` in `change-scala-version.sh` downloads the POM file of `commons-cli` but doesn't download the JAR file, leading the build failure.

This PR also adds `commons-cli` to the dependencies explicitly because HiveThriftServer depends on it.
### Why are the changes needed?

Expect to fix the build failure with Scala 2.13.

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

No.

### How was this patch tested?

I confirmed that build successfully finishes with Scala 2.13 on my laptop.
```
find ~/.m2 -name commons-cli -exec rm -rf {} \;
find ~/.ivy2 -name commons-cli -exec rm -rf {} \;
find ~/.cache/ -name commons-cli -exec rm -rf {} \; // For Linux
find ~/Library/Caches -name commons-cli -exec rm -rf {} \; // For macOS

dev/change-scala-version 2.13
./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pdocker-integration-tests -Pkubernetes-integration-tests -Pspark-ganglia-lgpl -Pscala-2.13 clean compile test:compile
```

Closes #31862 from sarutak/commons-cli.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 12:31:50 +09:00
gengjiaan 569fb133d0 [SPARK-33602][SQL] Group exception messages in execution/datasources
### What changes were proposed in this pull request?
This PR group exception messages in `/core/src/main/scala/org/apache/spark/sql/execution/datasources`.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31757 from beliefer/SPARK-33602.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 14:04:02 +00:00
Wenchen Fan 9f7b0a035b [SPARK-34758][SQL] Simplify Analyzer.resolveLiteralFunction
### What changes were proposed in this pull request?

This PR simplifies `Analyzer.resolveLiteralFunction` to always create the `Alias`. The caller side will remove the `Alias` if it's not necessary.

### Why are the changes needed?

code simplification.

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

no

### How was this patch tested?

existing tests

Closes #31844 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-17 21:26:44 +09:00
Wenchen Fan bf4570b43d [SPARK-34749][SQL] Simplify ResolveCreateNamedStruct
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31808 and simplifies its fix to one line (excluding comments).

### Why are the changes needed?

code simplification

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

no

### How was this patch tested?

N/A

Closes #31843 from cloud-fan/simplify.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-17 21:21:54 +09:00
ulysses-you 48637a9d43 [SPARK-34766][SQL] Do not capture maven config for views
### What changes were proposed in this pull request?

Skip capture maven repo config for views.

### Why are the changes needed?

Due to the bad network, we always use the thirdparty maven repo to run test. e.g.,
```
build/sbt "test:testOnly *SQLQueryTestSuite" -Dspark.sql.maven.additionalRemoteRepositories=xxxxx
```

It's failed with such error msg
```
[info] - show-tblproperties.sql *** FAILED *** (128 milliseconds)
[info] show-tblproperties.sql
[info] Expected "...rredTempViewNames [][]", but got "...rredTempViewNames [][
[info] view.sqlConfig.spark.sql.maven.additionalRemoteRepositories xxxxx]" Result did not match for query #6
[info] SHOW TBLPROPERTIES view (SQLQueryTestSuite.scala:464)
```

It's not necessary to capture the maven config to view since it's a session level config.
 

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

No.

### How was this patch tested?

manual test pass
```
build/sbt "test:testOnly *SQLQueryTestSuite" -Dspark.sql.maven.additionalRemoteRepositories=xxx
```

Closes #31856 from ulysses-you/skip-maven-config.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-17 20:12:18 +08:00
HyukjinKwon 385f1e8f5d [SPARK-34768][SQL] Respect the default input buffer size in Univocity
### What changes were proposed in this pull request?

This PR proposes to follow Univocity's input buffer.

### Why are the changes needed?

- Firstly, it's best to trust their judgement on the default values. Also 128 is too low.
- Default values arguably have more test coverage in Univocity.
- It will also fix https://github.com/uniVocity/univocity-parsers/issues/449
- ^ is a regression compared to Spark 2.4

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

No. In addition, It fixes a regression.

### How was this patch tested?

Manually tested, and added a unit test.

Closes #31858 from HyukjinKwon/SPARK-34768.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-17 19:55:49 +09:00
Wenchen Fan 1a4971d8a1 [SPARK-34770][SQL] InMemoryCatalog.tableExists should not fail if database doesn't exist
### What changes were proposed in this pull request?

This PR updates `InMemoryCatalog.tableExists` to return false if database doesn't exist, instead of failing. The new behavior is consistent with `HiveExternalCatalog` which is used in production, so this bug mostly only affects tests.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

a new test

Closes #31860 from cloud-fan/catalog.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 16:36:50 +08:00
Kent Yao 115f777cb0 [SPARK-21449][SQL][FOLLOWUP] Avoid log undesirable IllegalStateException when state close
### What changes were proposed in this pull request?

`TmpOutputFile` and `TmpErrOutputFile`  are registered in `o.a.h.u.ShutdownHookManager `during creatation. The `state.close()` will delete them if they are not null and try remove them from the `o.a.h.u.ShutdownHookManager` which causes IllegalStateException when we call it in our ShutdownHookManager too.
In this PR, we delete them ahead with a high priority hook in Spark and set them to null to bypass the deletion and canceling in `state.close()`

### Why are the changes needed?

W/ or w/o this PR, the deletion of these files is not affected, we just mute an undesirable error log here.

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

no, this is a follow-up

### How was this patch tested?

#### the undesirable gone
```scala
spark-sql> 21/03/16 18:41:31 ERROR Utils: Uncaught exception in thread shutdown-hook-0
java.lang.IllegalStateException: Shutdown in progress, cannot cancel a deleteOnExit
	at org.apache.hive.common.util.ShutdownHookManager.cancelDeleteOnExit(ShutdownHookManager.java:106)
	at org.apache.hadoop.hive.common.FileUtils.deleteTmpFile(FileUtils.java:861)
	at org.apache.hadoop.hive.ql.session.SessionState.deleteTmpErrOutputFile(SessionState.java:325)
	at org.apache.hadoop.hive.ql.session.SessionState.dropSessionPaths(SessionState.java:829)
	at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1585)
	at org.apache.hadoop.hive.cli.CliSessionState.close(CliSessionState.java:66)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1994)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$1(ShutdownHookManager.scala:188)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at scala.util.Try$.apply(Try.scala:213)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	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)
(python)  ✘ kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  cd ..
(python)  kentyaohulk  ~/Downloads/spark  tar zxf spark-3.2.0-SNAPSHOT-bin-20210316.tgz
(python)  kentyaohulk  ~/Downloads/spark  cd -
~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316
(python)  kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  bin/spark-sql --conf spark.local.dir=./local --conf spark.hive.exec.local.scratchdir=./local
21/03/16 18:42:15 WARN Utils: Your hostname, hulk.local resolves to a loopback address: 127.0.0.1; using 10.242.189.214 instead (on interface en0)
21/03/16 18:42:15 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
21/03/16 18:42:15 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
21/03/16 18:42:16 WARN SparkConf: Note that spark.local.dir will be overridden by the value set by the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone/kubernetes and LOCAL_DIRS in YARN).
21/03/16 18:42:18 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
21/03/16 18:42:18 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
21/03/16 18:42:19 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
21/03/16 18:42:19 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore kentyao127.0.0.1
Spark master: local[*], Application Id: local-1615891336877
spark-sql> %
```

#### and the deletion is still fine

```shell
kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316 
ls -al local
total 0
drwxr-xr-x   7 kentyao  staff  224  3 16 18:42 .
drwxr-xr-x  19 kentyao  staff  608  3 16 18:42 ..
drwx------   2 kentyao  staff   64  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e51
-rw-r--r--   1 kentyao  staff    0  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e51219959790473242539.pipeout
-rw-r--r--   1 kentyao  staff    0  3 16 18:42 16cc5238-e25e-4c0f-96ef-0c4bdecc7e518816377057377724129.pipeout
drwxr-xr-x   2 kentyao  staff   64  3 16 18:42 blockmgr-37a52ad2-eb56-43a5-8803-8f58d08fe9ad
drwx------   3 kentyao  staff   96  3 16 18:42 spark-101971df-f754-47c2-8764-58c45586be7e
 kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316  ls -al local
total 0
drwxr-xr-x   2 kentyao  staff   64  3 16 19:22 .
drwxr-xr-x  19 kentyao  staff  608  3 16 18:42 ..
 kentyaohulk  ~/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210316 
```

Closes #31850 from yaooqinn/followup.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-17 15:21:23 +08:00
Yuming Wang c234c5b5f1 [SPARK-34575][SQL] Push down limit through window when partitionSpec is empty
### What changes were proposed in this pull request?

Push down limit through `Window` when the partitionSpec of all window functions is empty and the same order is used. This is a real case from production:

![image](https://user-images.githubusercontent.com/5399861/109457143-3900c680-7a95-11eb-9078-806b041175c2.png)

This pr support 2 cases:
1. All window functions have same orderSpec:
   ```sql
   SELECT *, ROW_NUMBER() OVER(ORDER BY a) AS rn, RANK() OVER(ORDER BY a) AS rk FROM t1 LIMIT 5;
   == Optimized Logical Plan ==
   Window [row_number() windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame,          unboundedpreceding$(), currentrow$())) AS rn#4, rank(a#9L) windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#5], [a#9L ASC NULLS FIRST]
   +- GlobalLimit 5
      +- LocalLimit 5
         +- Sort [a#9L ASC NULLS FIRST], true
            +- Relation default.t1[A#9L,B#10L,C#11L] parquet
   ```
2. There is a window function with a different orderSpec:
   ```sql
   SELECT a, ROW_NUMBER() OVER(ORDER BY a) AS rn, RANK() OVER(ORDER BY b DESC) AS rk FROM t1 LIMIT 5;
   == Optimized Logical Plan ==
   Project [a#9L, rn#4, rk#5]
   +- Window [rank(b#10L) windowspecdefinition(b#10L DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#5], [b#10L DESC NULLS LAST]
      +- GlobalLimit 5
         +- LocalLimit 5
            +- Sort [b#10L DESC NULLS LAST], true
               +- Window [row_number() windowspecdefinition(a#9L ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#4], [a#9L ASC NULLS FIRST]
                  +- Project [a#9L, b#10L]
                     +- Relation default.t1[A#9L,B#10L,C#11L] parquet
   ```

### Why are the changes needed?

Improve query performance.

```scala
spark.range(500000000L).selectExpr("id AS a", "id AS b").write.saveAsTable("t1")
spark.sql("SELECT *, ROW_NUMBER() OVER(ORDER BY a) AS rowId FROM t1 LIMIT 5").show
```

Before this pr | After this pr
-- | --
![image](https://user-images.githubusercontent.com/5399861/109456919-c68fe680-7a94-11eb-89ca-67ec03267158.png) | ![image](https://user-images.githubusercontent.com/5399861/109456927-cd1e5e00-7a94-11eb-9866-d76b2665caea.png)

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

No.

### How was this patch tested?

Unit test.

Closes #31691 from wangyum/SPARK-34575.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 07:16:10 +00:00
Gengliang Wang 143303147b [SPARK-34742][SQL] ANSI mode: Abs throws exception if input is out of range
### What changes were proposed in this pull request?

For the following cases, ABS should throw exceptions since the results are out of the range of the result data types in ANSI mode.
```
SELECT abs(${Int.MinValue});
SELECT abs(${Long.MinValue});
```
### Why are the changes needed?

Better ANSI compliance

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

Yes, Abs throws an exception if input is out of range in ANSI mode

### How was this patch tested?

Unit test

Closes #31836 from gengliangwang/ansiAbs.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 06:57:25 +00:00
Terry Kim 387d866244 [SPARK-34699][SQL] 'CREATE OR REPLACE TEMP VIEW USING' should uncache correctly
### What changes were proposed in this pull request?

This PR proposes:
  1. `CREATE OR REPLACE TEMP VIEW USING` should use `TemporaryViewRelation` to store temp views.
  2. By doing #1, it fixes the issue where the temp view being replaced is not uncached.

### Why are the changes needed?

This is a part of an ongoing work to wrap all the temporary views with `TemporaryViewRelation`: [SPARK-34698](https://issues.apache.org/jira/browse/SPARK-34698).

This also fixes a bug where the temp view being replaced is not uncached.

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

Yes, the temp view being replaced with `CREATE OR REPLACE TEMP VIEW USING` is correctly uncached if the temp view is cached.

### How was this patch tested?

Added new tests.

Closes #31825 from imback82/create_temp_view_using.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 06:04:07 +00:00
Wenchen Fan af553735b1 [SPARK-34504][SQL] Avoid unnecessary resolving of SQL temp views for DDL commands
### What changes were proposed in this pull request?

For DDL commands like DROP VIEW, they don't really need to resolve the view (parse and analyze the view SQL text), they just need to get the view metadata.

This PR fixes the rule `ResolveTempViews` to only resolve the temp view for `UnresolvedRelation`. This also fixes a bug for DROP VIEW, as previously it tried to resolve the view and failed to drop invalid views.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

new test

Closes #31853 from cloud-fan/view-resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-17 11:16:51 +08:00
Wenchen Fan cef6650048 Revert "[SPARK-33428][SQL] Conv UDF use BigInt to avoid Long value overflow"
This reverts commit 5f9a7fea06.
2021-03-16 13:56:50 +08:00
Cheng Su bb05dc91f0 [SPARK-34729][SQL][FOLLOWUP] Broadcast nested loop join to use executeTake instead of execute
### What changes were proposed in this pull request?

This is a followup minor change from https://github.com/apache/spark/pull/31821#discussion_r594110622 , where we change from using `execute()` to `executeTake()`. Performance-wise there's no difference. We are just using a different API to be aligned with code path of `Dataset`.

### Why are the changes needed?

To align with other code paths in SQL/Dataset.

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

No.

### How was this patch tested?

Existing unit tests same as https://github.com/apache/spark/pull/31821 .

Closes #31845 from c21/join-followup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 19:48:45 -07:00
Kent Yao 202529ef23 [SPARK-21449][SPARK-23745][SQL] add ShutdownHook to cloes HiveClient's SessionState to delete residual dirs
### What changes were proposed in this pull request?

We initialized a Hive `SessionState` to interact with the external hive metastore server but left it behind after we finished.

We should close the metastore client explicitly in case of connection leaks with HMS
and we should trigger the `SessionState` to close itself to clean the residual dirs to fix issues reported by SPARK-21449 and SPARK-23745.

`hive.downloaded.resources.dir` contains transient files, such as UDF jars, it will not be used anymore after spark applications exit.

### Why are the changes needed?

1. prevent potential metastore client leak

2. clean `hive.downloaded.resources.dir`

```
    DOWNLOADED_RESOURCES_DIR("hive.downloaded.resources.dir", "${system:java.io.tmpdir}" + File.separator + "${hive.session.id}_resources", "Temporary local directory for added resources in the remote file system."),

```
### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

passing jenkins and verify locally

Closes #31833 from yaooqinn/SPARK-21449-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-16 10:37:40 +08:00
Dongjoon Hyun 0a70dff066 [MINOR][SQL] Remove unused variable in NewInstance.constructor
### What changes were proposed in this pull request?

This PR removes one unused variable in `NewInstance.constructor`.

### Why are the changes needed?

This looks like a variable for debugging at the initial commit of SPARK-23584 .
- 1b08c4393c (diff-2a36e31684505fd22e2d12a864ce89fd350656d716a3f2d7789d2cdbe38e15fbR461)

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31838 from dongjoon-hyun/minor-object.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 18:49:54 -07:00
Max Gekk 9809a2f1c5 [SPARK-34739][SQL] Support add/subtract of a year-month interval to/from a timestamp
### What changes were proposed in this pull request?
Support `timestamp +/- year-month interval`. In the PR, I propose to introduce new binary expression `TimestampAddYMInterval` similarly to `DateAddYMInterval`. It invokes new method `timestampAddMonths` from `DateTimeUtils` by passing a timestamp as an offset in microseconds since the epoch, amount of months from the giveb year-month interval, and the time zone ID in which the operation is performed. The `timestampAddMonths()` method converts the input microseconds to a local timestamp, adds months to it, and converts the results back to an instant in microseconds at the given time zone.

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over timestamps and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31832 from MaxGekk/timestamp-add-year-month-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-15 14:36:12 +03:00
Dongjoon Hyun 363a7f0722 [SPARK-34743][SQL][TESTS] ExpressionEncoderSuite should use deepEquals when we expect array of array
### What changes were proposed in this pull request?

This PR aims to make `ExpressionEncoderSuite` to use `deepEquals` instead of `equals` when `input` is `array of array`.

This comparison code itself was added by SPARK-11727 at Apache Spark 1.6.0.

### Why are the changes needed?

Currently, the interpreted mode fails for `array of array` because the following line is used.
```
Arrays.equals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]])
```

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

No. This is a test-only PR.

### How was this patch tested?

Pass the existing CIs.

Closes #31837 from dongjoon-hyun/SPARK-34743.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-15 02:30:54 -07:00
Wenchen Fan be888b27ed [SPARK-34639][SQL] Always remove unnecessary Alias in Analyzer.resolveExpression
### What changes were proposed in this pull request?

In `Analyzer.resolveExpression`, we have a parameter to decide if we should remove unnecessary `Alias` or not. This is over complicated and we can always remove unnecessary `Alias`.

This PR simplifies this part and removes the parameter.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31758 from cloud-fan/resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-15 09:22:36 +00:00
Cheng Su a0f3b72e1c [SPARK-34729][SQL] Faster execution for broadcast nested loop join (left semi/anti with no condition)
### What changes were proposed in this pull request?

For `BroadcastNestedLoopJoinExec` left semi and left anti join without condition. If we broadcast left side. Currently we check whether every row from broadcast side has a match or not by [iterating broadcast side a lot of time](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L256-L275). This is unnecessary and very inefficient when there's no condition, as we only need to check whether stream side is empty or not. Create this PR to add the optimization. This can boost the affected query execution performance a lot.

In addition, create a common method `getMatchedBroadcastRowsBitSet()` shared by several methods.
Refactor `defaultJoin()` to move
* left semi and left anti join related logic to `leftExistenceJoin`
* existence join related logic to `existenceJoin`.

After this, `defaultJoin()` holds logic only for outer join (left outer, right outer and full outer), which is much easier to read from my own opinion.

### Why are the changes needed?

Improve the affected query performance a lot.
Test with a simple query by modifying `JoinBenchmark.scala` locally:

```
val N = 20 << 20
val M = 1 << 4
val dim = broadcast(spark.range(M).selectExpr("id as k"))
val df = dim.join(spark.range(N), Seq.empty, "left_semi")
df.noop()
```

See >30x run time improvement. Note the stream side is only `spark.range(N)`. For complicated query with non-trivial stream side, the saving would be much more.

```
Running benchmark: broadcast nested loop left semi join
  Running case: broadcast nested loop left semi join optimization off
  Stopped after 2 iterations, 3163 ms
  Running case: broadcast nested loop left semi join optimization on
  Stopped after 5 iterations, 366 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
broadcast nested loop left semi join:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------------------
broadcast nested loop left semi join optimization off           1568           1582          19         13.4          74.8       1.0X
broadcast nested loop left semi join optimization on              46             73          18        456.0           2.2      34.1X
```

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

No.

### How was this patch tested?

Added unit test in `ExistenceJoinSuite.scala`.

Closes #31821 from c21/nested-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:51:36 -07:00
yangjie01 e757091820 [SPARK-34722][CORE][SQL][TEST] Clean up deprecated API usage related to JUnit4
### What changes were proposed in this pull request?
The main change of this pr as follows:

- Use `org.junit.Assert.assertThrows(String, Class, ThrowingRunnable)` method instead of  `ExpectedException.none()`
- Use `org.hamcrest.MatcherAssert.assertThat()` method instead of   `org.junit.Assert.assertThat(T, org.hamcrest.Matcher<? super T>)`

### Why are the changes needed?
Clean up deprecated API usage

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31815 from LuciferYang/SPARK-34722.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:33:03 -07:00
Max Gekk 7aaed76125 [SPARK-34737][SQL] Cast input float to double in TIMESTAMP_SECONDS
### What changes were proposed in this pull request?
In the PR, I propose to cast the input float to double in the `SecondsToTimestamp` expression in the same way as in the `Cast` expression.

### Why are the changes needed?
To have the same results from `CAST(<float> AS TIMESTAMP)` and from `TIMESTAMP_SECONDS`:
```sql
spark-sql> SELECT CAST(16777215.0f AS TIMESTAMP);
1970-07-14 07:20:15
spark-sql> SELECT TIMESTAMP_SECONDS(16777215.0f);
1970-07-14 07:20:14.951424
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> SELECT TIMESTAMP_SECONDS(16777215.0f);
1970-07-14 07:20:15
```

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *DateExpressionsSuite"
```

Closes #31831 from MaxGekk/adjust-SecondsToTimestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-15 10:05:59 +09:00
Max Gekk e0a1399bd7 [SPARK-34727][SQL] Fix discrepancy in casting float to timestamp
### What changes were proposed in this pull request?
In non-ANSI mode, casting float to timestamp has different implementation for codegen on and off.

Codegen on:
1. Multiply float input by MICROS_PER_SECOND
2. Cast resulting float value to long

Codegen off:
1. CAST float input to double input
2. Multiply double input by MICROS_PER_SECOND
3. Cast resulting double value to long

In the PR, I propose to align to non-codegen code, and cast input float to double in codegen.

### Why are the changes needed?
This fixes the issue which is demonstrated by the code:
```sql
spark-sql> CREATE TEMP VIEW v1 AS SELECT 16777215.0f AS f;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:15
spark-sql> CACHE TABLE v1;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:14.951424
```
The result from the cached view **1970-07-14 07:20:14.951424** is different from un-cached view **1970-07-14 07:20:15**.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above outputs the same timestamp for the cached view:
```sql
spark-sql> CACHE TABLE v1;
spark-sql> SELECT * FROM v1;
1.6777215E7
spark-sql> SELECT CAST(f AS TIMESTAMP) FROM v1;
1970-07-14 07:20:15
```

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *CastSuite"
```

Closes #31819 from MaxGekk/fix-float-to-timestamp.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-14 11:29:54 +09:00
Liang-Chi Hsieh 86baa36eeb [SPARK-34723][SQL] Correct parameter type for subexpression elimination under whole-stage
### What changes were proposed in this pull request?

This patch proposes to fix incorrect parameter type for subexpression elimination under whole-stage.

### Why are the changes needed?

If the parameter is a byte array, the subexpression elimination under wholestage codegen will use incorrect parameter type and cause compile error. Although Spark can automatically fallback to interpreted mode, we should fix it.

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

No

### How was this patch tested?

Manually test with customer application. Unit test.

Closes #31814 from viirya/SPARK-34723.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-13 00:05:41 -08:00
Max Gekk 4f1e434ec5 [SPARK-34721][SQL] Support add/subtract of a year-month interval to/from a date
### What changes were proposed in this pull request?
Support `date +/- year-month interval`. In the PR, I propose to re-use existing code from the `AddMonths` expression, and extract it to the common base class `AddMonthsBase`. That base class is used in new expression `DateAddYMInterval` and in the existing one `AddMonths` (the `add_months` function).

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over dates and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/110914390-5f412480-8327-11eb-9f8b-e92e73c0b9cd.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *ColumnExpressionSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
```

Closes #31812 from MaxGekk/date-add-year-month-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 14:35:56 +00:00
Dongjoon Hyun 9a7977933f [SPARK-34724][SQL] Fix Interpreted evaluation by using getMethod instead of getDeclaredMethod
### What changes were proposed in this pull request?

This bug was introduced by SPARK-23583 at Apache Spark 2.4.0.

This PR aims to use `getMethod` instead of `getDeclaredMethod`.
```scala
- obj.getClass.getDeclaredMethod(functionName, argClasses: _*)
+ obj.getClass.getMethod(functionName, argClasses: _*)
```

### Why are the changes needed?

`getDeclaredMethod` does not search the super class's method. To invoke `GenericArrayData.toIntArray`, we need to use `getMethod` because it's declared at the super class `ArrayData`.

```
[info] - encode/decode for array of int: [I74655d03 (interpreted path) *** FAILED *** (14 milliseconds)
[info]   Exception thrown while decoding
[info]   Converted: [0,1000000020,3,0,ffffff850000001f,4]
[info]   Schema: value#680
[info]   root
[info]   -- value: array (nullable = true)
[info]       |-- element: integer (containsNull = false)
[info]
[info]
[info]   Encoder:
[info]   class[value[0]: array<int>] (ExpressionEncoderSuite.scala:578)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Assertions.fail(Assertions.scala:949)
[info]   at org.scalatest.Assertions.fail$(Assertions.scala:945)
[info]   at org.scalatest.funsuite.AnyFunSuite.fail(AnyFunSuite.scala:1563)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:578)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$4(PlanTest.scala:50)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.withSQLConf(ExpressionEncoderSuite.scala:118)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$3(PlanTest.scala:50)
...
[info]   Cause: java.lang.RuntimeException: Error while decoding: java.lang.NoSuchMethodException: org.apache.spark.sql.catalyst.util.GenericArrayData.toIntArray()
[info] mapobjects(lambdavariable(MapObject, IntegerType, false, -1), assertnotnull(lambdavariable(MapObject, IntegerType, false, -1)), input[0, array<int>, true], None).toIntArray
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:186)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:576)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
```

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

This causes a runtime exception when we use the interpreted mode.

### How was this patch tested?

Pass the modified unit test case.

Closes #31816 from dongjoon-hyun/SPARK-34724.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-12 21:30:46 +09:00
Kousuke Saruta 03dd33cc98 [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute, AttributeReference and Alias don't quote qualified names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `sql` method in the following classes which take qualified names don't quote the qualified names properly.

* UnresolvedAttribute
* AttributeReference
* Alias

One instance caused by this issue is reported in SPARK-34626.
```
UnresolvedAttribute("a" :: "b" :: Nil).sql
`a.b` // expected: `a`.`b`
```
And other instances are like as follows.
```
UnresolvedAttribute("a`b"::"c.d"::Nil).sql
a`b.`c.d` // expected: `a``b`.`c.d`

AttributeReference("a.b", IntegerType)(qualifier = "c.d"::Nil).sql
c.d.`a.b` // expected: `c.d`.`a.b`

Alias(AttributeReference("a", IntegerType)(), "b.c")(qualifier = "d.e"::Nil).sql
`a` AS d.e.`b.c` // expected: `a` AS `d.e`.`b.c`
```

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

New test.

Closes #31754 from sarutak/fix-qualified-names.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 02:58:46 +00:00
Max Gekk cebe2be221 [SPARK-34718][SQL] Assign pretty names to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to override the `typeName()` method in `YearMonthIntervalType` and `DayTimeIntervalType`, and assign them names according to the ANSI SQL standard:
<img width="836" alt="Screenshot 2021-03-11 at 17 29 04" src="https://user-images.githubusercontent.com/1580697/110802854-a54aa980-828f-11eb-956d-dd4fbf14aa72.png">
but keep the type name as singular according existing naming convention for other types.

### Why are the changes needed?
To improve Spark SQL user experience, and have readable types in error messages.

### Does this PR introduce _any_ user-facing change?
Should not since the types has not been released yet.

### How was this patch tested?
By running the modified tests:
```
$ build/sbt "test:testOnly *ExpressionTypeCheckingSuite"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z windowFrameCoercion.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z literals.sql"
```

Closes #31810 from MaxGekk/interval-types-name.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:55:12 -08:00
Angerszhuuuu badca975af [SPARK-34712][SQL][TESTS] Refactor UT about hive build in version, avoid to change every time when upgrade hive version
### What changes were proposed in this pull request?
Use HiveUtils.buildinHiveVersion to replace correspoding Ut about hive version

### Why are the changes needed?
Refactor UT about hive build in version, avoid to change every time when upgrade hive version

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

### How was this patch tested?
Not need

Closes #31807 from AngersZhuuuu/SPARK-34712.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:52:29 -08:00
Wenchen Fan 6a42b633bf [SPARK-34713][SQL] Fix group by CreateStruct with ExtractValue
### What changes were proposed in this pull request?

This is a bug caused by https://issues.apache.org/jira/browse/SPARK-31670 . We remove the `Alias` when resolving column references in grouping expressions, which breaks `ResolveCreateNamedStruct`

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

new tests

Closes #31808 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 09:21:58 -08:00
Max Gekk d7bb327aee [SPARK-34695][SQL] Fix long overflow in conversion of minimum duration to microseconds
### What changes were proposed in this pull request?
In the PR, I propose to especially handle the amount of seconds `-9223372036855` in `IntervalUtils. durationToMicros()`. Starting from the amount (any durations with the second field < `-9223372036855`), input durations cannot fit to `Long` in the conversion to microseconds. For example, the amount of microseconds = `Long.MinValue = -9223372036854775808` can be represented in two forms:
1. seconds = -9223372036854, nanoAdjustment = -775808, or
2. seconds = -9223372036855, nanoAdjustment = +224192

And the method `Duration.ofSeconds()` produces the last form but such form causes overflow while converting `-9223372036855` seconds to microseconds.

In the PR, I propose to convert the second form to the first one if the second field of input duration is equal to `-9223372036855`.

### Why are the changes needed?
The changes fix the issue demonstrated by the code:
```scala
scala> durationToMicros(microsToDuration(Long.MinValue))
java.lang.ArithmeticException: long overflow
  at java.lang.Math.multiplyExact(Math.java:892)
  at org.apache.spark.sql.catalyst.util.IntervalUtils$.durationToMicros(IntervalUtils.scala:782)
  ... 49 elided
```
The `durationToMicros()` method cannot handle valid output of `microsToDuration()`.

### Does this PR introduce _any_ user-facing change?
Should not since new interval types has not been released yet.

### How was this patch tested?
By running new UT from `IntervalUtilsSuite`.

Closes #31799 from MaxGekk/fix-min-duration.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:21:15 +00:00
ulysses-you 744a73df9e [SPARK-34538][SQL] Hive Metastore support filter by not-in
### What changes were proposed in this pull request?

Add `Not(In)` and `Not(InSet)` pattern when convert filter to metastore.

### Why are the changes needed?

`NOT IN` is a useful condition to prune partition, it would be better to support it.

Technically, we can convert `c not in(x,y)` to `c != x and c != y`, then push it to metastore.

Avoid metastore overflow and respect the config `spark.sql.hive.metastorePartitionPruningInSetThreshold`, `Not(InSet)` won't push to metastore if it's value exceeds the threshold.

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

No.

### How was this patch tested?

Add test.

Closes #31646 from ulysses-you/SPARK-34538.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:19:47 +00:00
Cheng Su da086a8ea3 [SPARK-34702][SQL] Avoid unnecessary code generation in JoinCodegenSupport.genBuildSideVars
### What changes were proposed in this pull request?

As a followup from code review in https://github.com/apache/spark/pull/31736#discussion_r588134104 , for `JoinCodegenSupport.genBuildSideVars`, we only need to generate build side variables with default values for LEFT OUTER and RIGHT OUTER join, but not for other join types (i.e. LEFT SEMI and LEFT ANTI). Create this PR to clean up the code.

In addition, change `BroadcastNestedLoopJoinExec` unit test to cover both whole stage code-gen enabled and disabled. Harden the unit tests to exercise all code paths.

### Why are the changes needed?

Avoid unnecessary code generation.

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

No.

### How was this patch tested?

Existing unit tests.
* BHJ and SHJ inner join is covered in `InnerJoinSuite.scala`
* BHJ and SHJ left outer and right outer join are covered in `OuterJoinSuite.scala`
* BHJ and SHJ left semi, left anti and existence join are covered in `ExistenceJoinSuite.scala`

Closes #31802 from c21/join-codegen-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 13:29:45 +00:00
Kousuke Saruta fa1cf5c207 [SPARK-34697][SQL] Allow DESCRIBE FUNCTION and SHOW FUNCTIONS explain about || (string concatenation operator)
### What changes were proposed in this pull request?

This PR fixes the behavior of `SHOW FUNCTIONS` and `DESCRIBE FUNCTION` for the `||` operator.
The result of `SHOW FUNCTIONS` doesn't contains `||` and `DESCRIBE FUNCTION ||` says `Function: || not found.` even though `||` is supported.

### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

Confirmed manually with the following commands.
```
spark-sql> DESCRIBE FUNCTION ||;
Function: ||
Usage: expr1 || expr2 - Returns the concatenation of `expr1` and `expr2`.

spark-sql> SHOW FUNCTIONS;
!
!=
%

...

|
||
~
```

Closes #31800 from sarutak/fix-describe-concat-pipe.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-11 22:11:26 +09:00
Max Gekk 9d3d25bca4 [SPARK-34677][SQL] Support the +/- operators over ANSI SQL intervals
### What changes were proposed in this pull request?
Extend the `Add`, `Subtract` and `UnaryMinus` expression to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the `overflow` exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
To conform to the ANSI SQL standard which defines `-/+` over intervals:
<img width="822" alt="Screenshot 2021-03-09 at 21 59 22" src="https://user-images.githubusercontent.com/1580697/110523128-bd50ea80-8122-11eb-9982-782da0088d27.png">

### Does this PR introduce _any_ user-facing change?
Should not since new types have not been released yet.

### How was this patch tested?
By running new tests in the test suites:
```
$ build/sbt "test:testOnly *ArithmeticExpressionSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31789 from MaxGekk/add-subtruct-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 10:08:43 +00:00
Dongjoon Hyun 5c4d8f9538 [SPARK-34696][SQL][TESTS] Fix CodegenInterpretedPlanTest to generate correct test cases
### What changes were proposed in this pull request?

SPARK-23596 added `CodegenInterpretedPlanTest` at Apache Spark 2.4.0 in a wrong way because `withSQLConf` depends on the execution time `SQLConf.get` instead of `test` function declaration time. So, the following code executes the test twice without controlling the `CodegenObjectFactoryMode`. This PR aims to fix it correct and introduce a new function `testFallback`.

```scala
trait CodegenInterpretedPlanTest extends PlanTest {

   override protected def test(
       testName: String,
       testTags: Tag*)(testFun: => Any)(implicit pos: source.Position): Unit = {
     val codegenMode = CodegenObjectFactoryMode.CODEGEN_ONLY.toString
     val interpretedMode = CodegenObjectFactoryMode.NO_CODEGEN.toString

     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) {
       super.test(testName + " (codegen path)", testTags: _*)(testFun)(pos)
     }
     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) {
       super.test(testName + " (interpreted path)", testTags: _*)(testFun)(pos)
     }
   }
 }
```

### Why are the changes needed?

1. We need to use like the following.
```scala
super.test(testName + " (codegen path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { testFun })(pos)
super.test(testName + " (interpreted path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) { testFun })(pos)
```

2. After we fix this behavior with the above code, several test cases including SPARK-34596 and SPARK-34607 fail because they didn't work at both `CODEGEN` and `INTERPRETED` mode. Those test cases only work at `FALLBACK` mode. So, inevitably, we need to introduce `testFallback`.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31766 from dongjoon-hyun/SPARK-34596-SPARK-34607.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:41:49 -08:00
Cheng Su 14ad7afa1a [MINOR][SQL] Remove unnecessary extend from BroadcastHashJoinExec
### What changes were proposed in this pull request?

This is just a minor fix. `HashJoin` already extends `JoinCodegenSupport`. So we don't need `CodegenSupport` here for `BroadcastHashJoinExec`. Submitted separately as a PR here per https://github.com/apache/spark/pull/31802#discussion_r592066686 .

### Why are the changes needed?

Clean up code.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #31805 from c21/bhj-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:38:53 -08:00
Cheng Su 9aa8f06313 [SPARK-34711][SQL][TESTS] Exercise code-gen enable/disable code paths for SHJ in join test suites
### What changes were proposed in this pull request?

Per comment in https://github.com/apache/spark/pull/31802#discussion_r592068440 , we would like to exercise whole stage code-gen enabled and disabled code paths in join unit test suites. This is for better test coverage of shuffled hash join.

### Why are the changes needed?

Better test coverage.

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

No.

### How was this patch tested?

Existing and added unit tests here.

Closes #31806 from c21/test-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:34:09 -08:00
Anton Okolnychyi 72263797bc [SPARK-34457][SQL] DataSource V2: Add default null ordering to SortDirection
### What changes were proposed in this pull request?

This PR adds a default null ordering to public `SortDirection` to match the Catalyst behavior.

### Why are the changes needed?

The SQL standard does not define the default null ordering for a sort direction. That's why it is up to a query engine to assign one. We need to standardize this in our public connector expressions to avoid ambiguity. That's why I propose to match the behavior in our Catalyst expressions.

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

Yes, it affects unreleased connector expression API.

### How was this patch tested?

Existing tests.

Closes #31580 from aokolnychyi/spark-34457.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:47:31 +00:00
Terry Kim 2a6e68e1f7 [SPARK-34546][SQL] AlterViewAs.query should be analyzed during the analysis phase, and AlterViewAs should invalidate the cache
### What changes were proposed in this pull request?

This PR proposes the following:
   * `AlterViewAs.query` is currently analyzed in the physical operator `AlterViewAsCommand`, but it should be analyzed during the analysis phase.
   *  When `spark.sql.legacy.storeAnalyzedPlanForView` is set to true, store `TermporaryViewRelation` which wraps the analyzed plan, similar to #31273.
   *  Try to uncache the view you are altering.

### Why are the changes needed?

Analyzing a plan should be done in the analysis phase if possible.

Not uncaching the view (existing behavior) seems like a bug since the cache may not be used again.

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

Yes, now the view can be uncached if it's already cached.

### How was this patch tested?

Added new tests around uncaching.

The existing tests such as `SQLViewSuite` should cover the analysis changes.

Closes #31652 from imback82/alter_view_child.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:31:40 +00:00
Andy Grove fc182f7e7f [SPARK-34682][SQL] Use PrivateMethodTester instead of reflection
### Why are the changes needed?
SPARK-34682 was merged prematurely. This PR implements feedback from the review. I wasn't sure whether I should create a new JIRA or not.

### Does this PR introduce _any_ user-facing change?
No. Just improves the test.

### How was this patch tested?
Updated test.

Closes #31798 from andygrove/SPARK-34682-follow-up.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 12:08:31 -08:00
Andy Grove fd4843803c [SPARK-34682][SQL] Fix regression in canonicalization error check in CustomShuffleReaderExec
### What changes were proposed in this pull request?
There is a regression in 3.1.1 compared to 3.0.2 when checking for a canonicalized plan when executing CustomShuffleReaderExec.

The regression was caused by the call to `sendDriverMetrics` which happens before the check and will always fail if the plan is canonicalized.

### Why are the changes needed?
This is a regression in a useful error check.

### Does this PR introduce _any_ user-facing change?
No. This is not an error that a user would typically see, as far as I know.

### How was this patch tested?
I tested this change locally by making a distribution from this PR branch. Before fixing the regression I saw:

```
java.util.NoSuchElementException: key not found: numPartitions
```

After fixing this regression I saw:

```
java.lang.IllegalStateException: operating on canonicalized plan
```

Closes #31793 from andygrove/SPARK-34682.

Lead-authored-by: Andy Grove <andygrove73@gmail.com>
Co-authored-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-10 20:48:00 +09:00
Cheng Su a916690dd9 [SPARK-34681][SQL] Fix bug for full outer shuffled hash join when building left side with non-equal condition
### What changes were proposed in this pull request?

For full outer shuffled hash join with building hash map on left side, and having non-equal condition, the join can produce wrong result.

The root cause is `boundCondition` in `HashJoin.scala` always assumes the left side row is `streamedPlan` and right side row is `buildPlan` ([streamedPlan.output ++ buildPlan.output](https://github.com/apache/spark/blob/branch-3.1/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L141)). This is valid assumption, except for full outer + build left case.

The fix is to correct `boundCondition` in `HashJoin.scala` to handle full outer + build left case properly. See reproduce in https://issues.apache.org/jira/browse/SPARK-32399?focusedCommentId=17298414&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17298414 .

### Why are the changes needed?

Fix data correctness bug.

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

No.

### How was this patch tested?

Changed the test in `OuterJoinSuite.scala` to cover full outer shuffled hash join.
Before this change, the unit test `basic full outer join using ShuffledHashJoin` in `OuterJoinSuite.scala` is failed.

Closes #31792 from c21/join-bugfix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 22:55:27 -08:00
Wenchen Fan 48377d5bd9 [SPARK-34676][SQL][TEST] TableCapabilityCheckSuite should not inherit all tests from AnalysisSuite
### What changes were proposed in this pull request?

Fixes a mistake in `TableCapabilityCheckSuite`, which runs some tests repeatedly.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

N/A

Closes #31788 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 09:02:31 -08:00
Kousuke Saruta 2fd85174e9 [SPARK-34603][SQL] Support ADD ARCHIVE and LIST ARCHIVES command
### What changes were proposed in this pull request?

This PR adds `ADD ARCHIVE` and `LIST ARCHIVES` commands to SQL and updates relevant documents.
SPARK-33530 added `addArchive` and `listArchives` to `SparkContext` but it's not supported yet to add/list archives with SQL.

### Why are the changes needed?

To complement features.

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

No.

### How was this patch tested?

Added new test and confirmed the generated HTML from the updated documents.

Closes #31721 from sarutak/sql-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-09 21:28:35 +09:00
Terry Kim cf8bbc184b [SPARK-34152][SQL][FOLLOW-UP] Global temp view's identifier should be correctly stored
### What changes were proposed in this pull request?

This PR proposed to fix a bug introduced in #31273 (https://github.com/apache/spark/pull/31273/files#r589494855).
### Why are the changes needed?

This fixes a bug where global temp view's database name was not passed correctly.

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

Yes, now the global temp view's database is correctly stored.

### How was this patch tested?

Added a new test that catches the bug.

Closes #31783 from imback82/SPARK-34152-bug-fix.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 12:08:08 +00:00
Amandeep Sharma a9c11896a5 [SPARK-34649][SQL][DOCS] org.apache.spark.sql.DataFrameNaFunctions.replace() fails for column name having a dot
### What changes were proposed in this pull request?

Use resolved attributes instead of data-frame fields for replacing values.

### Why are the changes needed?

dataframe.na.replace() does not work for column having a dot in the name

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

None

### How was this patch tested?

Added unit tests for the same

Closes #31769 from amandeep-sharma/master.

Authored-by: Amandeep Sharma <happyaman91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:47:01 +00:00
Cheng Su b5b198516c [SPARK-34620][SQL] Code-gen broadcast nested loop join (inner/cross)
### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not have code-gen, and we can potentially boost the CPU performance for this operator if we add code-gen for it. https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html also showed the evidence in one fork.

The codegen for `BroadcastNestedLoopJoinExec` shared some code with `HashJoin`, and the interface `JoinCodegenSupport` is created to hold those common logic. This PR is only supporting inner and cross join. Other join types will be added later in followup PRs.

Example query and generated code:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 =!= $"k2").explain("codegen")
```

```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, Inner, NOT ((k1#2L + 1) = k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 038 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 039 */
/* 040 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 041 */
/* 042 */       long bnlj_value_4 = -1L;
/* 043 */
/* 044 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 045 */
/* 046 */       boolean bnlj_value_3 = false;
/* 047 */       bnlj_value_3 = bnlj_value_4 == bnlj_value_1;
/* 048 */       boolean bnlj_value_2 = false;
/* 049 */       bnlj_value_2 = !(bnlj_value_3);
/* 050 */       if (!(false || !bnlj_value_2))
/* 051 */       {
/* 052 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 053 */
/* 054 */         range_mutableStateArray_0[3].reset();
/* 055 */
/* 056 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 057 */
/* 058 */         range_mutableStateArray_0[3].write(1, bnlj_value_1);
/* 059 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 060 */
/* 061 */       }
/* 062 */     }
/* 063 */
/* 064 */   }
/* 065 */
/* 066 */   private void initRange(int idx) {
/* 067 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 068 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 069 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 070 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 071 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 072 */     long partitionEnd;
/* 073 */
/* 074 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 075 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 076 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 077 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 078 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 079 */     } else {
/* 080 */       range_nextIndex_0 = st.longValue();
/* 081 */     }
/* 082 */     range_batchEnd_0 = range_nextIndex_0;
/* 083 */
/* 084 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 085 */     .multiply(step).add(start);
/* 086 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 087 */       partitionEnd = Long.MAX_VALUE;
/* 088 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 089 */       partitionEnd = Long.MIN_VALUE;
/* 090 */     } else {
/* 091 */       partitionEnd = end.longValue();
/* 092 */     }
/* 093 */
/* 094 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 095 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 096 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 097 */     if (range_numElementsTodo_0 < 0) {
/* 098 */       range_numElementsTodo_0 = 0;
/* 099 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 100 */       range_numElementsTodo_0++;
/* 101 */     }
/* 102 */   }
/* 103 */
/* 104 */   protected void processNext() throws java.io.IOException {
/* 105 */     // initialize Range
/* 106 */     if (!range_initRange_0) {
/* 107 */       range_initRange_0 = true;
/* 108 */       initRange(partitionIndex);
/* 109 */     }
/* 110 */
/* 111 */     while (true) {
/* 112 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 113 */         long range_nextBatchTodo_0;
/* 114 */         if (range_numElementsTodo_0 > 1000L) {
/* 115 */           range_nextBatchTodo_0 = 1000L;
/* 116 */           range_numElementsTodo_0 -= 1000L;
/* 117 */         } else {
/* 118 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 119 */           range_numElementsTodo_0 = 0;
/* 120 */           if (range_nextBatchTodo_0 == 0) break;
/* 121 */         }
/* 122 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 123 */       }
/* 124 */
/* 125 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 126 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 127 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 128 */
/* 129 */         // common sub-expressions
/* 130 */
/* 131 */         bnlj_doConsume_0(range_value_0);
/* 132 */
/* 133 */         if (shouldStop()) {
/* 134 */           range_nextIndex_0 = range_value_0 + 1L;
/* 135 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 136 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 137 */           return;
/* 138 */         }
/* 139 */
/* 140 */       }
/* 141 */       range_nextIndex_0 = range_batchEnd_0;
/* 142 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 143 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 144 */       range_taskContext_0.killTaskIfInterrupted();
/* 145 */     }
/* 146 */   }
/* 147 */
/* 148 */ }
```

### Why are the changes needed?

Improve query CPU performance. Added a micro benchmark query in `JoinBenchmark.scala`.
Saw 1x of run time improvement:

```
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off          62922          63052         184          0.3        3000.3       1.0X
broadcast nested loop join wholestage on           30946          30972          26          0.7        1475.6       2.0X
```

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

No.

### How was this patch tested?

* Added unit test in `WholeStageCodegenSuite.scala`, and existing unit tests for `BroadcastNestedLoopJoinExec`.
* Updated golden files for several TCPDS query plans, as whole stage code-gen for `BroadcastNestedLoopJoinExec` is triggered.
* Updated `JoinBenchmark-jdk11-results.txt ` and `JoinBenchmark-results.txt` with new benchmark result. Followed previous benchmark PRs - https://github.com/apache/spark/pull/27078 and https://github.com/apache/spark/pull/26003 to use same type of machine:

```
Amazon AWS EC2
type: r3.xlarge
region: us-west-2 (Oregon)
OS: Linux
```

Closes #31736 from c21/nested-join-exec.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:45:43 +00:00
Takeshi Yamamuro 43b23fd132 [SPARK-33498][SQL][TESTS][FOLLOWUP] Remove SQLConf.withExistingConf in CastSuite
### What changes were proposed in this pull request?

This PR intends to remove unnecessary `SQLConf.withExistingConf` in `CastSuite`; since we've remove `ParVector ` in #31775, we no longer need to copy SQL configs into each thread env.

### Why are the changes needed?

Clean up the code.

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

No.

### How was this patch tested?

Run the existing tests.

Closes #31785 from maropu/UpdateCastSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-08 23:52:43 -08:00
Max Gekk 6a7701a5be [SPARK-34663][SQL][TESTS] Test year-month and day-time intervals in UDF
### What changes were proposed in this pull request?
Added new tests to `UDFSuite` to check `java.time.Period`/`java.time.Duration` in UDF as input parameters as well as UDF results.

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *UDFSuite"
```

Closes #31779 from MaxGekk/interval-udf.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:58:14 +00:00
Max Gekk 4ea27787bf [SPARK-34666][SQL][TESTS] Test DayTimeIntervalType and YearMonthIntervalType as ordered and atomic types
### What changes were proposed in this pull request?
Add `DayTimeIntervalType` and `YearMonthIntervalType` to `DataTypeTestUtils.ordered`/`atomicTypes`, and implement values generation of those types in `LiteralGenerator`/`RandomDataGenerator`. In this way, the types will be tested automatically in:
1. ArithmeticExpressionSuite:
    - "function least"
    - "function greatest"
2. PredicateSuite
    - "BinaryComparison consistency check"
    - "AND, OR, EqualTo, EqualNullSafe consistency check"
3. ConditionalExpressionSuite
    - "if"
4. RandomDataGeneratorSuite
    - "Basic types"
5. CastSuite
    - "null cast"
    - "up-cast"
    - "SPARK-27671: cast from nested null type in struct"
6. OrderingSuite
    - "GenerateOrdering with DayTimeIntervalType"
    - "GenerateOrdering with YearMonthIntervalType"
7. PredicateSuite
    - "IN with different types"
8. UnsafeRowSuite
    - "calling get(ordinal, datatype) on null columns"
9. SortSuite
    - "sorting on YearMonthIntervalType ..."
    - "sorting on DayTimeIntervalType ..."

### Why are the changes needed?
To improve test coverage.

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

### How was this patch tested?
By running the affected test suites.

Closes #31782 from MaxGekk/test-interval-as-atomic.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:42:59 +00:00
allisonwang-db c32cac4cd6 [SPARK-34627][SQL] Use FunctionIdentifier in UnresolvedTableValuedFunction
### What changes were proposed in this pull request?
This PR updates UnresolvedTableValuedFunction's name to be a FunctionIdentifier instead of a string.

### Why are the changes needed?
To make UnresolvedTableValuedFunction consistent with UnresolvedFunction that uses FunctionIdentifier as the function name.

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

### How was this patch tested?
Unit test.

Closes #31749 from allisonwang-db/spark-34627.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 05:27:02 +00:00
Swinky 02e74b298a [SPARK-34598][SQL] RewritePredicateSubquery Rule must not update Filters without subqueries
### What changes were proposed in this pull request?
RewritePredicateSubquery Optimizer Rule must not update Filters without subqueries.

Following is one such example.

```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery ===
 Project [a#0]                                                        Project [a#0]
!+- Filter (((a#0 > 1) OR (b#1 > 2)) AND ((c#2 > 1) AND (d#3 > 2)))   +- Filter ((((a#0 > 1) OR (b#1 > 2)) AND (c#2 > 1)) AND (d#3 > 2))
    +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]                       +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]
```

### Why are the changes needed?
minor change.

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

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

Closes #31712 from Swinky/rewritePredicateFix.

Authored-by: Swinky <mannswinky@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 09:01:49 +00:00
Max Gekk e10bf64769 [SPARK-34615][SQL] Support java.time.Period as an external type of the year-month interval type
### What changes were proposed in this pull request?
In the PR, I propose to extend Spark SQL API to accept [`java.time.Period`](https://docs.oracle.com/javase/8/docs/api/java/time/Period.html) as an external type of recently added new Catalyst type - `YearMonthIntervalType` (see #31614). The Java class `java.time.Period` has similar semantic to ANSI SQL year-month interval type, and it is the most suitable to be an external type for `YearMonthIntervalType`. In more details:
1. Added `PeriodConverter` which converts `java.time.Period` instances to/from internal representation of the Catalyst type `YearMonthIntervalType` (to `Int` type). The `PeriodConverter` object uses new methods of `IntervalUtils`:
    - `periodToMonths()` converts the input period to the total length in months. If this period is too large to fit `Int`, the method throws the exception `ArithmeticException`. **Note:** _the input period has "days" precision, the method just ignores the days unit._
    - `monthToPeriod()` obtains a `java.time.Period` representing a number of months.
2. Support new type `YearMonthIntervalType` in `RowEncoder` via the methods `createDeserializerForPeriod()` and `createSerializerForJavaPeriod()`.
3. Extended the Literal API to construct literals from `java.time.Period` instances.

### Why are the changes needed?
1. To allow users parallelization of `java.time.Period` collections, and construct year-month interval columns. Also to collect such columns back to the driver side.
2. This will allow to write tests in other sub-tasks of SPARK-27790.

### Does this PR introduce _any_ user-facing change?
The PR extends existing functionality. So, users can parallelize instances of the `java.time.Duration` class and collect them back:

```scala
scala> val ds = Seq(java.time.Period.ofYears(10).withMonths(2)).toDS
ds: org.apache.spark.sql.Dataset[java.time.Period] = [value: yearmonthinterval]

scala> ds.collect
res0: Array[java.time.Period] = Array(P10Y2M)
```

### How was this patch tested?
- Added a few tests to `CatalystTypeConvertersSuite` to check conversion from/to `java.time.Period`.
- Checking row encoding by new tests in `RowEncoderSuite`.
- Making literals of `YearMonthIntervalType` are tested in `LiteralExpressionSuite`.
- Check collecting by `DatasetSuite` and `JavaDatasetSuite`.
- New tests in `IntervalUtilsSuites` to check conversions `java.time.Period` <-> months.

Closes #31765 from MaxGekk/java-time-period.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 08:33:09 +00:00
yangjie01 43f355b5f2 [SPARK-34597][SQL] Replaces ParquetFileReader.readFooter with ParquetFileReader.open and getFooter
### What changes were proposed in this pull request?
`ParquetFileReader.readFooter` related methods has been identified as `Deprecated` and `Apache Parquet` suggests replace it with the combination of `ParquetFileReader.open() and getFooter()` methods.

This PR introduces the `ParquetFooterReader` utility class due to some repetitive code patterns when read parquet file footer.

### Why are the changes needed?
Cleanup deprecated API usage.

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31711 from LuciferYang/parquet-read-footer.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-07 23:38:40 -08:00
Peter Toth ab8a9a0ceb [SPARK-34545][SQL] Fix issues with valueCompare feature of pyrolite
### What changes were proposed in this pull request?

pyrolite 4.21 introduced and enabled value comparison by default (`valueCompare=true`) during object memoization and serialization: https://github.com/irmen/Pyrolite/blob/pyrolite-4.21/java/src/main/java/net/razorvine/pickle/Pickler.java#L112-L122
This change has undesired effect when we serialize a row (actually `GenericRowWithSchema`) to be passed to python: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L60. A simple example is that
```
new GenericRowWithSchema(Array(1.0, 1.0), StructType(Seq(StructField("_1", DoubleType), StructField("_2", DoubleType))))
```
and
```
new GenericRowWithSchema(Array(1, 1), StructType(Seq(StructField("_1", IntegerType), StructField("_2", IntegerType))))
```
are currently equal and the second instance is replaced to the short code of the first one during serialization.

### Why are the changes needed?
The above can cause nasty issues like the one in https://issues.apache.org/jira/browse/SPARK-34545 description:

```
>>> from pyspark.sql.functions import udf
>>> from pyspark.sql.types import *
>>>
>>> def udf1(data_type):
        def u1(e):
            return e[0]
        return udf(u1, data_type)
>>>
>>> df = spark.createDataFrame([((1.0, 1.0), (1, 1))], ['c1', 'c2'])
>>>
>>> df = df.withColumn("c3", udf1(DoubleType())("c1"))
>>> df = df.withColumn("c4", udf1(IntegerType())("c2"))
>>>
>>> df.select("c3").show()
+---+
| c3|
+---+
|1.0|
+---+

>>> df.select("c4").show()
+---+
| c4|
+---+
|  1|
+---+

>>> df.select("c3", "c4").show()
+---+----+
| c3|  c4|
+---+----+
|1.0|null|
+---+----+
```
This is because during serialization from JVM to Python `GenericRowWithSchema(1.0, 1.0)` (`c1`) is memoized first and when `GenericRowWithSchema(1, 1)` (`c2`) comes next, it is replaced to some short code of the `c1` (instead of serializing `c2` out) as they are `equal()`. The python functions then runs but the return type of `c4` is expected to be `IntegerType` and if a different type (`DoubleType`) comes back from python then it is discarded: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L108-L113

After this PR:
```
>>> df.select("c3", "c4").show()
+---+---+
| c3| c4|
+---+---+
|1.0|  1|
+---+---+
```

### Does this PR introduce _any_ user-facing change?
Yes, fixes a correctness issue.

### How was this patch tested?
Added new UT + manual tests.

Closes #31682 from peter-toth/SPARK-34545-fix-row-comparison.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-07 19:12:42 -06:00
wangguangxin.cn 9ec8696f11 [SPARK-34634][SQL] ResolveReferences.dedupRight should handle ScriptTransformation
### What changes were proposed in this pull request?
When we do self join with transform in a CTE, spark will throw AnalysisException.

A simple way to reproduce is

```
create temporary view t as select * from values 0, 1, 2 as t(a);

WITH temp AS (
  SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t
)
SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b
```

before this patch, it throws

```
org.apache.spark.sql.AnalysisException: cannot resolve '`t1.b`' given input columns: [t1.b]; line 6 pos 41;
'Project ['t1.b]
+- 'Join Inner, ('t1.b = 't2.b)
   :- SubqueryAlias t1
   :  +- SubqueryAlias temp
   :     +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
   :        +- SubqueryAlias t
   :           +- Project [a#1]
   :              +- SubqueryAlias t
   :                 +- LocalRelation [a#1]
   +- SubqueryAlias t2
      +- SubqueryAlias temp
         +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
            +- SubqueryAlias t
               +- Project [a#1]
                  +- SubqueryAlias t
                     +- LocalRelation [a#1]
```

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

### How was this patch tested?
Add a UT

Closes #31752 from WangGuangxin/selfjoin-with-transform.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-07 15:53:52 +09:00
Yuming Wang 616c818e7c [SPARK-34628][SQL] Remove GlobalLimit operator if its child max rows not larger than limit number
### What changes were proposed in this pull request?

This pr remove `GlobalLimit` operator if its child max rows not larger than limit number. For example:
```
val testRelation = LocalRelation.fromExternalRows(Seq("a".attr.int, "b".attr.int, "c".attr.int), 1.to(10).map(_ => Row(1, 2, 3)) )
val query = GlobalLimit(100, testRelation)
```
We can remove this `GlobalLimit`.

### Why are the changes needed?

Further optimize the query.

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

No.

### How was this patch tested?

Unit test.

Closes #31750 from wangyum/SPARK-34628.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 09:54:15 -08:00
helloman 1a9722420e [SPARK-34595][SQL] DPP support RLIKE
### What changes were proposed in this pull request?
This pr make DPP support RLIKE expression:

```sql
SELECT date_id, product_id FROM fact_sk f
JOIN dim_store s
ON f.store_id = s.store_id WHERE s.country RLIKE  '[DE|US]'
```
 ### 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 #31722 from chaojun-zhang/SPARK-34595.

Authored-by: helloman <zcj23085@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-06 20:30:20 +09:00
Angerszhuuuu 654f19dfd7 [SPARK-34621][SQL] Unify output of ShowCreateTableAsSerdeCommand and ShowCreateTableCommand
### What changes were proposed in this pull request?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

### Why are the changes needed?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

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

### How was this patch tested?

Closes #31737 from AngersZhuuuu/SPARK-34621.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:44:07 +00:00
suqilong ca326c4bb3 [SPARK-22748][SQL] Analyze __grouping__id as a literal function
### What changes were proposed in this pull request?

This PR intends to refactor the logic to resolve `__grouping_id` in the `Analyzer`; it moves the logic from `ResolveFunctions` to `ResolveReferences` (`resolveLiteralFunction`).

The original author of this PR is sqlwindspeaker (#30781).

Closes #30781.

### Why are the changes needed?

Code refactoring.

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

No.

### How was this patch tested?

Added tests in `AnalysisSuite`.

Closes #31751 from maropu/SPARK-22748.

Authored-by: suqilong <suqilong@qiyi.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:40:58 +00:00
Wenchen Fan dc78f337cb [SPARK-34609][SQL] Unify resolveExpressionBottomUp and resolveExpressionTopDown
### What changes were proposed in this pull request?

It's a bit confusing to see `resolveExpressionBottomUp` and `resolveExpressionTopDown`, which provide similar functionalities but with different tree traverse order. It turns out that the real difference between these 2 methods is: which attributes should the columns be resolved to? `resolveExpressionTopDown` resolves columns using output attributes of the plan children, `resolveExpressionBottomUp` resolves columns using output attributes of the plan itself.

This PR unifies `resolveExpressionBottomUp` and `resolveExpressionTopDown` and put the common logic in a new method, and let `resolveExpressionBottomUp` and `resolveExpressionTopDown` just call the new method. This PR also renames `resolveExpressionBottomUp` and `resolveExpressionTopDown` to make the difference clear.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31728 from cloud-fan/resolve.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 05:59:15 +00:00
Angerszhuuuu 979b9bcf5d [SPARK-34608][SQL] Remove unused output of AddJarCommand
### What changes were proposed in this pull request?
Remove unused output of AddJarCommand, keep consistence and clean

### Why are the changes needed?
Keep consistence and clean

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

### How was this patch tested?
Not need

Closes #31725 from AngersZhuuuu/SPARK-34608.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-04 21:19:07 -08:00
ulysses-you 43aacd5069 [SPARK-34613][SQL] Fix view does not capture disable hint config
### What changes were proposed in this pull request?

Add allow list to capture sql config for view.

### Why are the changes needed?

Spark use origin text sql to store view then capture and store sql config into view metadata.

Capture config will skip some config with some prefix, e.g. `spark.sql.optimizer.` but unfortunately `spark.sql.optimizer.disableHints` is start with `spark.sql.optimizer.`.

We need a allow list to help capture the config.

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

Yes bug fix.

### How was this patch tested?

Add test.

Closes #31732 from ulysses-you/SPARK-34613.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 12:19:30 +08:00
Kent Yao 814d81c1e5 [SPARK-34376][SQL] Support regexp as a SQL function
### What changes were proposed in this pull request?

We have equality in `SqlBase.g4` for `RLIKE: 'RLIKE' | 'REGEXP';`
We seemed to miss adding` REGEXP` as a SQL function just like` RLIKE`

### Why are the changes needed?

symmetry and beauty
This is also a builtin  function in Hive, we can reduce the migration pain for those users

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

yes new regexp function as an alias as rlike

### How was this patch tested?

new tests

Closes #31488 from yaooqinn/SPARK-34376.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-05 12:09:28 +09:00
Takeshi Yamamuro dbce74d39d [SPARK-34607][SQL] Add Utils.isMemberClass to fix a malformed class name error on jdk8u
### What changes were proposed in this pull request?

This PR intends to fix a bug of `objects.NewInstance` if a user runs Spark on jdk8u and a given `cls` in `NewInstance` is a deeply-nested inner class, e.g.,.
```
  object OuterLevelWithVeryVeryVeryLongClassName1 {
    object OuterLevelWithVeryVeryVeryLongClassName2 {
      object OuterLevelWithVeryVeryVeryLongClassName3 {
        object OuterLevelWithVeryVeryVeryLongClassName4 {
          object OuterLevelWithVeryVeryVeryLongClassName5 {
            object OuterLevelWithVeryVeryVeryLongClassName6 {
              object OuterLevelWithVeryVeryVeryLongClassName7 {
                object OuterLevelWithVeryVeryVeryLongClassName8 {
                  object OuterLevelWithVeryVeryVeryLongClassName9 {
                    object OuterLevelWithVeryVeryVeryLongClassName10 {
                      object OuterLevelWithVeryVeryVeryLongClassName11 {
                        object OuterLevelWithVeryVeryVeryLongClassName12 {
                          object OuterLevelWithVeryVeryVeryLongClassName13 {
                            object OuterLevelWithVeryVeryVeryLongClassName14 {
                              object OuterLevelWithVeryVeryVeryLongClassName15 {
                                object OuterLevelWithVeryVeryVeryLongClassName16 {
                                  object OuterLevelWithVeryVeryVeryLongClassName17 {
                                    object OuterLevelWithVeryVeryVeryLongClassName18 {
                                      object OuterLevelWithVeryVeryVeryLongClassName19 {
                                        object OuterLevelWithVeryVeryVeryLongClassName20 {
                                          case class MalformedNameExample2(x: Int)
                                        }}}}}}}}}}}}}}}}}}}}
```

The root cause that Kris (rednaxelafx) investigated is as follows (Kudos to Kris);

The reason why the test case above is so convoluted is in the way Scala generates the class name for nested classes. In general, Scala generates a class name for a nested class by inserting the dollar-sign ( `$` ) in between each level of class nesting. The problem is that this format can concatenate into a very long string that goes beyond certain limits, so Scala will change the class name format beyond certain length threshold.

For the example above, we can see that the first two levels of class nesting have class names that look like this:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$OuterLevelWithVeryVeryVeryLongClassName2$
```
If we leave out the fact that Scala uses a dollar-sign ( `$` ) suffix for the class name of the companion object, `OuterLevelWithVeryVeryVeryLongClassName1`'s full name is a prefix (substring) of `OuterLevelWithVeryVeryVeryLongClassName2`.

But if we keep going deeper into the levels of nesting, you'll find names that look like:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$OuterLevelWithVeryVeryVeryLongClassName11$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$OuterLevelWithVeryVeryVeryLongClassName14$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$OuterLevelWithVeryVeryVeryLongClassName17$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$OuterLevelWithVeryVeryVeryLongClassName20$
```
with a hash code in the middle and various levels of nesting omitted.

The `java.lang.Class.isMemberClass` method is implemented in JDK8u as:
http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/tip/src/share/classes/java/lang/Class.java#l1425
```
    /**
     * Returns {code true} if and only if the underlying class
     * is a member class.
     *
     * return {code true} if and only if this class is a member class.
     * since 1.5
     */
    public boolean isMemberClass() {
        return getSimpleBinaryName() != null && !isLocalOrAnonymousClass();
    }

    /**
     * Returns the "simple binary name" of the underlying class, i.e.,
     * the binary name without the leading enclosing class name.
     * Returns {code null} if the underlying class is a top level
     * class.
     */
    private String getSimpleBinaryName() {
        Class<?> enclosingClass = getEnclosingClass();
        if (enclosingClass == null) // top level class
            return null;
        // Otherwise, strip the enclosing class' name
        try {
            return getName().substring(enclosingClass.getName().length());
        } catch (IndexOutOfBoundsException ex) {
            throw new InternalError("Malformed class name", ex);
        }
    }
```
and the problematic code is `getName().substring(enclosingClass.getName().length())` -- if a class's enclosing class's full name is *longer* than the nested class's full name, this logic would end up going out of bounds.

The bug has been fixed in JDK9 by https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8057919 , but still exists in the latest JDK8u release. So from the Spark side we'd need to do something to avoid hitting this problem.

### Why are the changes needed?

Bugfix on jdk8u.

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

No.

### How was this patch tested?

Added tests.

Closes #31733 from maropu/SPARK-34607.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-05 08:59:30 +09:00
Max Gekk 17601e014c [SPARK-34605][SQL] Support java.time.Duration as an external type of the day-time interval type
### What changes were proposed in this pull request?
In the PR, I propose to extend Spark SQL API to accept [`java.time.Duration`](https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html) as an external type of recently added new Catalyst type - `DayTimeIntervalType` (see #31614). The Java class `java.time.Duration` has similar semantic to ANSI SQL day-time interval type, and it is the most suitable to be an external type for `DayTimeIntervalType`. In more details:
1. Added `DurationConverter` which converts `java.time.Duration` instances to/from internal representation of the Catalyst type `DayTimeIntervalType` (to `Long` type). The `DurationConverter` object uses new methods of `IntervalUtils`:
    - `durationToMicros()` converts the input duration to the total length in microseconds. If this duration is too large to fit `Long`, the method throws the exception `ArithmeticException`. **Note:** _the input duration has nanosecond precision, the method casts the nanos part to microseconds by dividing by 1000._
    - `microsToDuration()` obtains a `java.time.Duration` representing a number of microseconds.
2. Support new type `DayTimeIntervalType` in `RowEncoder` via the methods `createDeserializerForDuration()` and `createSerializerForJavaDuration()`.
3. Extended the Literal API to construct literals from `java.time.Duration` instances.

### Why are the changes needed?
1. To allow users parallelization of `java.time.Duration` collections, and construct day-time interval columns. Also to collect such columns back to the driver side.
2. This will allow to write tests in other sub-tasks of SPARK-27790.

### Does this PR introduce _any_ user-facing change?
The PR extends existing functionality. So, users can parallelize instances of the `java.time.Duration` class and collect them back:

```Scala
scala> val ds = Seq(java.time.Duration.ofDays(10)).toDS
ds: org.apache.spark.sql.Dataset[java.time.Duration] = [value: daytimeinterval]

scala> ds.collect
res0: Array[java.time.Duration] = Array(PT240H)
```

### How was this patch tested?
- Added a few tests to `CatalystTypeConvertersSuite` to check conversion from/to `java.time.Duration`.
- Checking row encoding by new tests in `RowEncoderSuite`.
- Making literals of `DayTimeIntervalType` are tested in `LiteralExpressionSuite`
- Check collecting by `DatasetSuite` and `JavaDatasetSuite`.

Closes #31729 from MaxGekk/java-time-duration.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 16:58:33 +00:00
yi.wu e7e016192f [SPARK-34482][SS] Correct the active SparkSession for StreamExecution.logicalPlan
### What changes were proposed in this pull request?

Set the active SparkSession to `sparkSessionForStream` and diable AQE & CBO before initializing the `StreamExecution.logicalPlan`.

### Why are the changes needed?

The active session should be `sparkSessionForStream`. Otherwise, settings like

6b34745cb9/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala (L332-L335)

wouldn't take effect if callers access them from the active SQLConf, e.g., the rule of `InsertAdaptiveSparkPlan`. Besides, unlike `InsertAdaptiveSparkPlan` (which skips streaming plan), `CostBasedJoinReorder` seems to have the chance to take effect theoretically.

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

No.

### How was this patch tested?

Tested manually. Before the fix, `InsertAdaptiveSparkPlan` would try to apply AQE on the plan(wouldn't take effect though). After this fix, the rule returns directly.

Closes #31600 from Ngone51/active-session-for-stream.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 22:41:11 +08:00
Angerszhuuuu 401e270c17 [SPARK-34567][SQL] CreateTableAsSelect should update metrics too
### What changes were proposed in this pull request?
For command `CreateTableAsSelect` we use `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand` to insert data.
We will update metrics of  `InsertIntoHiveTable`, `InsertIntoHadoopFsRelationCommand`  in `FileFormatWriter.write()`, but we only show CreateTableAsSelectCommand in WebUI SQL Tab.
We need to update `CreateTableAsSelectCommand`'s metrics too.

Before this PR:
![image](https://user-images.githubusercontent.com/46485123/109411226-81f44480-79db-11eb-99cb-b9686b15bf61.png)

After this PR:
![image](https://user-images.githubusercontent.com/46485123/109411232-8ae51600-79db-11eb-9111-3bea0bc2d475.png)

![image](https://user-images.githubusercontent.com/46485123/109905192-62aa2f80-7cd9-11eb-91f9-04b16c9238ae.png)

### Why are the changes needed?
Complete SQL Metrics

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

### How was this patch tested?
<!--
MT

Closes #31679 from AngersZhuuuu/SPARK-34567.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 20:42:47 +08:00
Gengliang Wang 2b1c170016 [SPARK-34614][SQL] ANSI mode: Casting String to Boolean should throw exception on parse error
### What changes were proposed in this pull request?

In ANSI mode, casting String to Boolean should throw an exception on parse error, instead of returning null

### Why are the changes needed?

For better ANSI compliance

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

Yes, in ANSI mode there will be an exception on parse failure of casting String value to Boolean type.

### How was this patch tested?

Unit tests.

Closes #31734 from gengliangwang/ansiCastToBoolean.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-03-04 19:04:16 +08:00
Shixiong Zhu 53e4dba7c4 [SPARK-34599][SQL] Fix the issue that INSERT INTO OVERWRITE doesn't support partition columns containing dot for DSv2
### What changes were proposed in this pull request?

`ResolveInsertInto.staticDeleteExpression` should use `UnresolvedAttribute.quoted` to create the delete expression so that we will treat the entire `attr.name` as a column name.

### Why are the changes needed?

When users use `dot` in a partition column name, queries like ```INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')``` is not working.

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

Without this test, the above query will throw
```
[info]   org.apache.spark.sql.AnalysisException: cannot resolve '`a.b`' given input columns: [a.b, c.d];
[info] 'OverwriteByExpression RelationV2[a.b#17, c.d#18] default.tbl, ('a.b <=> cast(a as string)), false
[info] +- Project [a.b#19, ansi_cast(col1#16 as string) AS c.d#20]
[info]    +- Project [cast(a as string) AS a.b#19, col1#16]
[info]       +- LocalRelation [col1#16]
```

With the fix, the query will run correctly.

### How was this patch tested?

The new added test.

Closes #31713 from zsxwing/SPARK-34599.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 15:12:53 +08:00
Angerszhuuuu db627107b7 [SPARK-34577][SQL] Fix drop/add columns to a dataset of DESCRIBE NAMESPACE
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the DESCRIBE NAMESPACE command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:

```
sql(s"CREATE NAMESPACE ns")
val description = sql(s"DESCRIBE NAMESPACE ns")
description.drop("name")
```

```
[info]   org.apache.spark.sql.AnalysisException: Resolved attribute(s) name#74 missing from name#25,value#26 in operator !Project [name#74]. Attribute(s) with the same name appear in the operation: name. Please check if the right attribute(s) are used.;
[info] !Project [name#74]
[info] +- LocalRelation [name#25, value#26]
```

### Does this PR introduce _any_ user-facing change?
After this change user `drop()/add()` works well.

### How was this patch tested?
Added UT

Closes #31705 from AngersZhuuuu/SPARK-34577.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-04 13:22:10 +08:00
Wenchen Fan 8f1eec4d13 [SPARK-34584][SQL] Static partition should also follow StoreAssignmentPolicy when insert into v2 tables
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/27597 and simply apply the fix in the v2 table insertion code path.

### Why are the changes needed?

bug fix

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

yes, now v2 table insertion with static partitions also follow StoreAssignmentPolicy.

### How was this patch tested?

moved the test from https://github.com/apache/spark/pull/27597 to the general test suite `SQLInsertTestSuite`, which covers DS v2, file source, and hive tables.

Closes #31726 from cloud-fan/insert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-04 11:29:34 +09:00
Erik Krogen 9d2d620e98 [SPARK-33084][SQL][TEST][FOLLOWUP] Add ResetSystemProperties trait to SQLQuerySuite to avoid clearing ivy.home
### What changes were proposed in this pull request?
Add the `ResetSystemProperties` trait to `SQLQuerySuite` so that system property changes made by any of the tests will not affect other suites/tests. Specifically, the system property changes made by `SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class` are targeted here (which sets and then clears `ivy.home`).

### Why are the changes needed?
PR #29966 added a new test case that adjusts the `ivy.home` system property to force Ivy to resolve an artifact from a custom location. At the end of the test, the value is cleared. Clearing the value meant that, if a custom value of `ivy.home` was configured externally, it would not apply for tests run after this test case.

### Does this PR introduce _any_ user-facing change?
No, this is only in tests.

### How was this patch tested?
Existing unit tests continue to pass, whether or not `spark.jars.ivySettings` is configured (which adjusts the behavior of Ivy w.r.t. handling of `ivy.home` and `ivy.default.ivy.user.dir` properties).

Closes #31694 from xkrogen/xkrogen-SPARK-33084-ivyhome-sysprop-followon.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-04 08:52:53 +09:00
Gengliang Wang 5aaab19685 [SPARK-34222][SQL][FOLLOWUP] Non-recursive implementation of buildBalancedPredicate
### What changes were proposed in this pull request?

Use a non-recursive implementation for the function buildBalancedPredicate
### Why are the changes needed?

For better performance.

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

No
### How was this patch tested?

Existing unit tests.
Also, a quick benchmark:
```
  test("buildBalancedPredicate") {
    val expressions = (1 to 1000).map(_ => Literal(true))
    val start = System.currentTimeMillis()
    buildBalancedPredicate(expressions, And)
    println(System.currentTimeMillis() - start)
  }
```
Before: 47ms
After: 4ms

Closes #31724 from gengliangwang/nonrecursive.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-03-04 01:01:28 +08:00
Karen Feng b01dd12805 [SPARK-34555][SQL] Resolve metadata output from DataFrame
### What changes were proposed in this pull request?

Add metadataOutput as a fallback to resolution.
Builds off https://github.com/apache/spark/pull/31654.

### Why are the changes needed?

The metadata columns could not be resolved via `df.col("metadataColName")` from the DataFrame API.

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

Yes, the metadata columns can now be resolved as described above.

### How was this patch tested?

Scala unit test.

Closes #31668 from karenfeng/spark-34555.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 22:07:41 +08:00
angerszhu 56edb8156f [SPARK-33474][SQL] Support TypeConstructed partition spec value
### What changes were proposed in this pull request?
Hive support type constructed value as partition spec value, spark should support too.

### Why are the changes needed?
 Support TypeConstructed partition spec value keep same with hive

### Does this PR introduce _any_ user-facing change?
Yes, user can use TypeConstruct value as partition spec value such as
```
CREATE TABLE t1(name STRING) PARTITIONED BY (part DATE)
INSERT INTO t1 PARTITION(part = date'2019-01-02') VALUES('a')

CREATE TABLE t2(name STRING) PARTITIONED BY (part TIMESTAMP)
INSERT INTO t2 PARTITION(part = timestamp'2019-01-02 11:11:11') VALUES('a')

CREATE TABLE t4(name STRING) PARTITIONED BY (part BINARY)
INSERT INTO t4 PARTITION(part = X'537061726B2053514C') VALUES('a')
```

### How was this patch tested?
Added UT

Closes #30421 from AngersZhuuuu/SPARK-33474.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-03 16:48:50 +09:00
Kent Yao 499f620037 [MINOR][SQL][DOCS] Fix some wrong default values in SQL tuning guide's AQE section
### What changes were proposed in this pull request?

spark.sql.adaptive.coalescePartitions.initialPartitionNum 200 -> (none)
spark.sql.adaptive.skewJoin.skewedPartitionFactor is 10 -> 5

### Why are the changes needed?

the wrong doc misguide people
### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

passing doc

Closes #31717 from yaooqinn/minordoc0.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 15:00:09 +09:00
Swinky 229d2e0554 [SPARK-34222][SQL] Enhance boolean simplification rule
### What changes were proposed in this pull request?
Enhance boolean simplification rule by handling following scenarios:
(((a && b) && a && (a && c))) => a && b && c)
(((a || b) || a || (a || c))) => a || b || c

### Why are the changes needed?
Minor improvement

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

### How was this patch tested?
Added UTs

Closes #31318 from Swinky/booleansimplification.

Authored-by: Swinky <mannswinky@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 05:25:28 +00:00
Angerszhuuuu 17f0e70fa0 [SPARK-34576][SQL] Fix drop/add columns to a dataset of DESCRIBE COLUMN
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the DESCRIBE COLUMN command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:

```
val tbl = "testcat.ns1.ns2.tbl"
sql(s"CREATE TABLE $tbl (c0 INT) USING _")
val description = sql(s"DESCRIBE TABLE $tbl c0")
description.drop("info_name")
```

```
[info]   org.apache.spark.sql.AnalysisException: Resolved attribute(s) info_name#74 missing from info_name#25,info_value#26 in operator !Project [info_name#74]. Attribute(s) with the same name appear in the operation: info_name. Please check if the right attribute(s) are used.;
[info] !Project [info_name#74]
[info] +- LocalRelation [info_name#25, info_value#26]
```

### Does this PR introduce _any_ user-facing change?
After this change user `drop()/add()` works well.

### How was this patch tested?
Added UT

Closes #31696 from AngersZhuuuu/SPARK-34576.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 12:51:30 +08:00
Max Gekk cd649e7aef [SPARK-27793][SQL] Add ANSI SQL day-time and year-month interval types
### What changes were proposed in this pull request?
In the PR, I propose to extend Catalyst's type system by two new types that conform to the SQL standard (see SQL:2016, section 4.6.3):
- `DayTimeIntervalType` represents the day-time interval type,
- `YearMonthIntervalType` for SQL year-month interval type.

This PR only adds the two new DataType implementations, and there will be more PRs as sub-tasks of SPARK-27790 to completely support the new ANSI interval types.

### Why are the changes needed?
Spark as it is today supports an INTERVAL datatype. However this type is of very limited use. Existing interval values cannot be compared with any other interval values, or persisted to storage. Spark users request to either implement new or expand existing built-in functions which produce some sort of measures for elapsed time, such as `DATEDIFF()`. Rather than work around the edges to fill the potholes of the existing INTERVAL data type, I would like to propose to deliver a proper ANSI compliant INTERVAL type that can be introduced with minimal incompatibility, is comparable and thus sortable, and can be persisted in tables.

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

### How was this patch tested?
1. By checking coding style via:
```
$ ./dev/scalastyle
$ ./dev/lint-java
```
2. Run the test for the default sizes:
```
$ build/sbt "test:testOnly *DataTypeSuite"
```

Closes #31614 from MaxGekk/day-time-interval-type.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 04:44:23 +00:00
Cheng Su 5362f08125 [SPARK-34593][SQL] Preserve broadcast nested loop join partitioning and ordering
### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not preserve `outputPartitioning` and `outputOrdering` right now. But it can preserve the streamed side partitioning and ordering when possible. This can help avoid shuffle and sort in later stage, if there's join and aggregation in the query. See example queries in added unit test in `JoinSuite.scala`.

In addition, fix a bunch of minor places in `BroadcastNestedLoopJoinExec.scala` for better style and readability.

### Why are the changes needed?

Avoid shuffle and sort for certain complicated query shape. Better query performance can be achieved.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite.scala`.

Closes #31708 from c21/nested-join.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-03 04:32:28 +00:00
Kris Mok ecf4811764 [SPARK-34596][SQL] Use Utils.getSimpleName to avoid hitting Malformed class name in NewInstance.doGenCode
### What changes were proposed in this pull request?

Use `Utils.getSimpleName` to avoid hitting `Malformed class name` error in `NewInstance.doGenCode`.

### Why are the changes needed?

On older JDK versions (e.g. JDK8u), nested Scala classes may trigger `java.lang.Class.getSimpleName` to throw an `java.lang.InternalError: Malformed class name` error.
In this particular case, creating an `ExpressionEncoder` on such a nested Scala class would create a `NewInstance` expression under the hood, which will trigger the problem during codegen.

Similar to https://github.com/apache/spark/pull/29050, we should use  Spark's `Utils.getSimpleName` utility function in place of `Class.getSimpleName` to avoid hitting the issue.

There are two other occurrences of `java.lang.Class.getSimpleName` in the same file, but they're safe because they're only guaranteed to be only used on Java classes, which don't have this problem, e.g.:
```scala
    // Make a copy of the data if it's unsafe-backed
    def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) =
      s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value"
    val genFunctionValue: String = lambdaFunction.dataType match {
      case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value)
      case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value)
      case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value)
      case _ => genFunction.value
    }
```
The Unsafe-* family of types are all Java types, so they're okay.

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

Fixes a bug that throws an error when using `ExpressionEncoder` on some nested Scala types, otherwise no changes.

### How was this patch tested?

Added a test case to `org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite`. It'll fail on JDK8u before the fix, and pass after the fix.

Closes #31709 from rednaxelafx/spark-34596-master.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 12:22:51 +09:00
Liang-Chi Hsieh 107766661a [SPARK-34548][SQL][FOLLOW-UP] Call toSeq to recover Scala 2.13 build in RemoveNoopUnion
### What changes were proposed in this pull request?

Call `toSeq` to fix Scala 2.13 build error.

### Why are the changes needed?

It is needed to fix 2.13 build error.

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

No.

### How was this patch tested?

Existing tests.

Closes #31716 from viirya/SPARK-34548-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-03 11:12:58 +09:00
Liang-Chi Hsieh bab9531134 [SPARK-34548][SQL] Remove unnecessary children from Union under Distince and Deduplicate
### What changes were proposed in this pull request?

This patch proposes to remove unnecessary children from Union under Distince and Deduplicate

### Why are the changes needed?

If there are any duplicate child of `Union` under `Distinct` and `Deduplicate`, it can be removed to simplify query plan.

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

No

### How was this patch tested?

Unit test

Closes #31656 from viirya/SPARK-34548.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-02 17:09:08 -08:00
Kent Yao 6093a78dbd [SPARK-34558][SQL] warehouse path should be qualified ahead of populating and use
### What changes were proposed in this pull request?

Currently, the warehouse path gets fully qualified in the caller side for creating a database, table, partition, etc. An unqualified path is populated into Spark and Hadoop confs, which leads to inconsistent API behaviors.  We should make it qualified ahead.

When the value is a relative path `spark.sql.warehouse.dir=lakehouse`, some behaviors become inconsistent, for example.

If the default database is absent at runtime, the app fails with

```java
Caused by: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:./lakehouse
	at org.apache.hadoop.fs.Path.initialize(Path.java:263)
	at org.apache.hadoop.fs.Path.<init>(Path.java:254)
	at org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:133)
	at org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:137)
	at org.apache.hadoop.hive.metastore.Warehouse.getWhRoot(Warehouse.java:150)
	at org.apache.hadoop.hive.metastore.Warehouse.getDefaultDatabasePath(Warehouse.java:163)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB_core(HiveMetaStore.java:636)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:655)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:431)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invokeInternal(RetryingHMSHandler.java:148)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:107)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:79)
	... 73 more
```

If the default database is present at runtime, the app can work with it, and if we create a database, it gets fully qualified, for example

```sql
spark-sql> create database test;
Time taken: 0.052 seconds
spark-sql> desc database test;
Database Name	test
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.2.0-SNAPSHOT-bin-20210226/lakehouse/test.db
Owner	kentyao
Time taken: 0.023 seconds, Fetched 4 row(s)
```

Another thing is that the log becomes nubilous, for example.

```logtalk
21/02/27 13:54:17 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir ('datalake').
21/02/27 13:54:17 INFO SharedState: Warehouse path is 'lakehouse'.
```

### Why are the changes needed?

fix bug and ambiguity
### Does this PR introduce _any_ user-facing change?

yes, the path now resolved with proper order - `warehouse->database->table->partition`

### How was this patch tested?

w/ ut added

Closes #31671 from yaooqinn/SPARK-34558.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 15:14:19 +00:00
kevincmchen 9e8547ca43 [SPARK-34498][SQL][TESTS] fix the remaining problems in #31560
### What changes were proposed in this pull request?

This is a followup of #31560,
In  #31560,  we added `JavaSimpleWritableDataSource ` and left some little problems like unused interface `SessionConfigSupport` 、 inconsistent schema between `JavaSimpleWritableDataSource ` and `SimpleWritableDataSource`.
This PR fixes the remaining problems in #31560.

### Why are the changes needed?

1. `SessionConfigSupport` in `JavaSimpleWritableDataSource ` and `SimpleWritableDataSource` is never used, so we don't need to implement it.
2. change the schema of `SimpleWritableDataSource`, to match `TestingV2Source`

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

NO

### How was this patch tested?

existing testsuites

Closes #31621 from kevincmchen/SPARK-34498.

Authored-by: kevincmchen <kevincmchen@tencent.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 15:08:23 +00:00
Karen Feng 2e54d68eb9 [SPARK-34547][SQL] Only use metadata columns for resolution as last resort
### What changes were proposed in this pull request?

Today, child expressions may be resolved based on "real" or metadata output attributes. We should prefer the real attribute during resolution if one exists.

### Why are the changes needed?

Today, attempting to resolve an expression when there is a "real" output attribute and a metadata attribute with the same name results in resolution failure. This is likely unexpected, as the user may not know about the metadata attribute.

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

Yes. Previously, the user would see an error message when resolving a column with the same name as a "real" output attribute and a metadata attribute as below:
```
org.apache.spark.sql.AnalysisException: Reference 'index' is ambiguous, could be: testcat.ns1.ns2.tableTwo.index, testcat.ns1.ns2.tableOne.index.; line 1 pos 71
at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:363)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveChildren(LogicalPlan.scala:107)
```

Now, resolution succeeds and provides the "real" output attribute.

### How was this patch tested?

Added a unit test.

Closes #31654 from karenfeng/fallback-resolve-metadata.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 17:27:13 +08:00
Amandeep Sharma 4bda3c0f02 [SPARK-34417][SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot
**What changes were proposed in this pull request?**

This PR fixes dataframe.na.fillMap() for column having a dot in the name as mentioned in [SPARK-34417](https://issues.apache.org/jira/browse/SPARK-34417).

Use resolved attributes of a column for replacing null values.

**Why are the changes needed?**
dataframe.na.fillMap() does not work for column having a dot in the name

**Does this PR introduce any user-facing change?**
None

**How was this patch tested?**
Added unit test for the same

Closes #31545 from amandeep-sharma/master.

Lead-authored-by: Amandeep Sharma <happyaman91@gmail.com>
Co-authored-by: Amandeep Sharma <amandeep.sharma@oracle.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-02 17:14:15 +08:00
Anton Okolnychyi 08a125761d [SPARK-34585][SQL] Remove no longer needed BatchWriteHelper
### What changes were proposed in this pull request?

As a follow-up to SPARK-34456, this PR removes `BatchWriteHelper` completely.

### Why are the changes needed?

These changes remove no longer used code.

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

No.

### How was this patch tested?

Existing tests.

Closes #31699 from aokolnychyi/spark-34585.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:58:18 +09:00
Dongjoon Hyun 4818847e87 [SPARK-34578][SQL][TESTS][TEST-MAVEN] Refactor ORC encryption tests and ignore ORC shim loaded by old Hadoop library
### What changes were proposed in this pull request?

1. This PR aims to ignore ORC encryption tests when ORC shim is loaded by old Hadoop library by some other tests. The test coverage is preserved by Jenkins SBT runs and GitHub Action jobs. This PR only aims to recover Maven Jenkins jobs.
2. In addition, this PR simplifies SBT testing by refactor the test config to `SparkBuild.scala/pom.xml` and remove `DedicatedJVMTest`. This will remove one GitHub Action job which was recently added for `DedicatedJVMTest` tag.

### Why are the changes needed?

Currently, Maven test fails when it runs in a batch mode because `HadoopShimsPre2_3$NullKeyProvider` is loaded.

**MVN COMMAND**
```
$ mvn test -pl sql/core --am -Dtest=none -DwildcardSuites=org.apache.spark.sql.execution.datasources.orc.OrcV1QuerySuite,org.apache.spark.sql.execution.datasources.orc.OrcEncryptionSuite
```

**BEFORE**
```
- Write and read an encrypted table *** FAILED ***
...
  Cause: 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) (localhost executor driver): java.lang.IllegalArgumentException: Unknown key pii
	at org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider.getCurrentKeyVersion(HadoopShimsPre2_3.java:71)
	at org.apache.orc.impl.WriterImpl.getKey(WriterImpl.java:871)
```

**AFTER**
```
OrcV1QuerySuite
...
OrcEncryptionSuite:
- Write and read an encrypted file !!! CANCELED !!!
  [] was empty org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider1b705f65 doesn't has the test keys. ORC shim is created with old Hadoop libraries (OrcEncryptionSuite.scala:39)
- Write and read an encrypted table !!! CANCELED !!!
  [] was empty org.apache.orc.impl.HadoopShimsPre2_3$NullKeyProvider22adeee1 doesn't has the test keys. ORC shim is created with old Hadoop libraries (OrcEncryptionSuite.scala:67)
```

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

No.

### How was this patch tested?

Pass the Jenkins Maven tests.

For SBT command,
- the test suite required a dedicated JVM (Before)
- the test suite doesn't require a dedicated JVM (After)
```
$ build/sbt "sql/testOnly *.OrcV1QuerySuite *.OrcEncryptionSuite"
...
[info] OrcV1QuerySuite
...
[info] - SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core (26 milliseconds)
[info] OrcEncryptionSuite:
[info] - Write and read an encrypted file (431 milliseconds)
[info] - Write and read an encrypted table (359 milliseconds)
[info] All tests passed.
[info] Passed: Total 35, Failed 0, Errors 0, Passed 35
```

Closes #31697 from dongjoon-hyun/SPARK-34578-TEST.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:52:27 +09:00
Chao Sun ce13dcc689 [SPARK-32703][SQL] Replace deprecated API calls from SpecificParquetRecordReaderBase
### What changes were proposed in this pull request?

Currently in `SpecificParquetRecordReaderBase` we use deprecated APIs in a few places from Parquet, such as `readFooter`, `ParquetInputSplit`, `new ParquetFileReader`, `filterRowGroups`, etc. This replaces these with the newer APIs. In specific this:
- Replaces `ParquetInputSplit` with `FileSplit`. We never use specific things in the former such as `rowGroupOffsets` so the swap is pretty simple.
- Removes `readFooter` calls by using `ParquetFileReader.open`
- Replace deprecated `ParquetFileReader` ctor with the newer API which takes `ParquetReadOptions`.
- Removes the unnecessary handling of case when `rowGroupOffsets` is not null. It seems this never happens.

### Why are the changes needed?

The aforementioned APIs were deprecated and is going to be removed at some point in future. This is to ensure better supportability.

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

No

### How was this patch tested?

This is a cleanup and relies on existing tests on the relevant code paths.

Closes #31667 from sunchao/SPARK-32703.

Lead-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:51:41 +09:00
Richard Penney 7d0743b493 [SPARK-33678][SQL] Product aggregation function
### Why is this change being proposed?
This patch adds support for a new "product" aggregation function in `sql.functions` which multiplies-together all values in an aggregation group.

This is likely to be useful in statistical applications which involve combining probabilities, or financial applications that involve combining cumulative interest rates, but is also a versatile mathematical operation of similar status to `sum` or `stddev`. Other users [have noted](https://stackoverflow.com/questions/52991640/cumulative-product-in-spark) the absence of such a function in current releases of Spark.

This function is both much more concise than an expression of the form `exp(sum(log(...)))`, and avoids awkward edge-cases associated with some values being zero or negative, as well as being less computationally costly.

### Does this PR introduce _any_ user-facing change?
No - only adds new function.

### How was this patch tested?
Built-in tests have been added for the new `catalyst.expressions.aggregate.Product` class and its invocation via the (scala) `sql.functions.product` function. The latter, and the PySpark wrapper have also been manually tested in spark-shell and pyspark sessions. The SparkR wrapper is currently untested, and may need separate validation (I'm not an "R" user myself).

An illustration of the new functionality, within PySpark is as follows:
```
import pyspark.sql.functions as pf, pyspark.sql.window as pw

df = sqlContext.range(1, 17).toDF("x")
win = pw.Window.partitionBy(pf.lit(1)).orderBy(pf.col("x"))

df.withColumn("factorial", pf.product("x").over(win)).show(20, False)
+---+---------------+
|x  |factorial      |
+---+---------------+
|1  |1.0            |
|2  |2.0            |
|3  |6.0            |
|4  |24.0           |
|5  |120.0          |
|6  |720.0          |
|7  |5040.0         |
|8  |40320.0        |
|9  |362880.0       |
|10 |3628800.0      |
|11 |3.99168E7      |
|12 |4.790016E8     |
|13 |6.2270208E9    |
|14 |8.71782912E10  |
|15 |1.307674368E12 |
|16 |2.0922789888E13|
+---+---------------+
```

Closes #30745 from rwpenney/feature/agg-product.

Lead-authored-by: Richard Penney <rwp@rwpenney.uk>
Co-authored-by: Richard Penney <rwpenney@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 16:51:07 +09:00
Gabriele Nizzoli b13a4b85d4 [SPARK-34573][SQL] Avoid global locking in SQLConf object for sqlConfEntries map
### What changes were proposed in this pull request?
In the `SQLConf` object, the `sqlConfEntries` map is globally synchronized (it is a Java `Collections.synchronizedMap`): any operation, including a get, will need to acquire the lock.

An example of this is calling the `DatatType.sameType` method. This will trigger a check on `SQLConf.get.caseSensitiveAnalysis`. So every time we compare two datatypes with sameType, we hit a lock.

To avoid having multiple tasks locking on this, a better approach would be to use a map that does not lock on read (like a `ConcurrentHashMap`). This map implementation does not lock on read, and on write it only locks the map partially. The only lock that happens is on write on the same map key.

### Why are the changes needed?
Multiple tasks performing any operation that directly or indirectly trigger a query to the `SQLConf.sqlConfEntries` map, will require acquiring a global lock on that map. Something as easy as calling `DataType.sameType(...)` would be locking on the global `sqlConfEntries` lock of the `Collections.synchronizedMap`.

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

### How was this patch tested?
No functionality change. Existing unit tests run normally.

Closes #31689 from gabrielenizzoli/SPARK-34573.

Authored-by: Gabriele Nizzoli <1545350+gabrielenizzoli@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-02 15:36:51 +09:00
Max Gekk 70f6267de6 [SPARK-34560][SQL] Generate unique output attributes in the SHOW TABLES logical node
### What changes were proposed in this pull request?
In the PR, I propose to generate unique attributes in the logical nodes of the `SHOW TABLES` command.

Also, this PR fixes similar issues in other logical nodes:
- ShowTableExtended
- ShowViews
- ShowTableProperties
- ShowFunctions
- ShowColumns
- ShowPartitions
- ShowNamespaces

### Why are the changes needed?
This fixes the issue which is demonstrated by the example below:
```scala
scala> val show1 = sql("SHOW TABLES IN ns1")
show1: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]

scala> val show2 = sql("SHOW TABLES IN ns2")
show2: org.apache.spark.sql.DataFrame = [namespace: string, tableName: string ... 1 more field]

scala> show1.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
|      ns1|     tbl1|      false|
+---------+---------+-----------+

scala> show2.show
+---------+---------+-----------+
|namespace|tableName|isTemporary|
+---------+---------+-----------+
|      ns2|     tbl2|      false|
+---------+---------+-----------+

scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
org.apache.spark.sql.AnalysisException: Column tableName#17 are ambiguous. It's probably because you joined several Datasets together, and some of these Datasets are the same. This column points to one of the Datasets but Spark is unable to figure out which one. Please alias the Datasets with different names via `Dataset.as` before joining them, and specify the column using qualified name, e.g. `df.as("a").join(df.as("b"), $"a.id" > $"b.id")`. You can also set spark.sql.analyzer.failAmbiguousSelfJoin to false to disable this check.
  at org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin$.apply(DetectAmbiguousSelfJoin.scala:157)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the example above works as expected:
```scala
scala> show1.join(show2).where(show1("tableName") =!= show2("tableName")).show
+---------+---------+-----------+---------+---------+-----------+
|namespace|tableName|isTemporary|namespace|tableName|isTemporary|
+---------+---------+-----------+---------+---------+-----------+
|      ns1|     tbl1|      false|      ns2|     tbl2|      false|
+---------+---------+-----------+---------+---------+-----------+
```

### How was this patch tested?
By running the new test:
```
$  build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
```

Closes #31675 from MaxGekk/fix-output-attrs.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-01 18:32:32 +00:00
Max Gekk 984ff396a2 [SPARK-34561][SQL] Fix drop/add columns from/to a dataset of v2 DESCRIBE TABLE
### What changes were proposed in this pull request?
In the PR, I propose to generate "stable" output attributes per the logical node of the `DESCRIBE TABLE` command.

### Why are the changes needed?
This fixes the issue demonstrated by the example:
```scala
val tbl = "testcat.ns1.ns2.tbl"
sql(s"CREATE TABLE $tbl (c0 INT) USING _")
val description = sql(s"DESCRIBE TABLE $tbl")
description.drop("comment")
```
The `drop()` method fails with the error:
```
org.apache.spark.sql.AnalysisException: Resolved attribute(s) col_name#102,data_type#103 missing from col_name#29,data_type#30,comment#31 in operator !Project [col_name#102, data_type#103]. Attribute(s) with the same name appear in the operation: col_name,data_type. Please check if the right attribute(s) are used.;
!Project [col_name#102, data_type#103]
+- LocalRelation [col_name#29, data_type#30, comment#31]

	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:51)
	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:50)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, `drop()`/`add()` works as expected:
```scala
description.drop("comment").show()
+---------------+---------+
|       col_name|data_type|
+---------------+---------+
|             c0|      int|
|               |         |
| # Partitioning|         |
|Not partitioned|         |
+---------------+---------+
```

### How was this patch tested?
1. Run new test:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *DataSourceV2SQLSuite"
```
2. Run existing test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31676 from MaxGekk/describe-table-drop-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-01 22:20:28 +08:00
Shixiong Zhu 62737e140c [SPARK-34556][SQL] Checking duplicate static partition columns should respect case sensitive conf
### What changes were proposed in this pull request?

This PR makes partition spec parsing respect case sensitive conf.

### Why are the changes needed?

When parsing the partition spec, Spark will call `org.apache.spark.sql.catalyst.parser.ParserUtils.checkDuplicateKeys` to check if there are duplicate partition column names in the list. But this method is always case sensitive and doesn't detect duplicate partition column names when using different cases.

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

Yep. This prevents users from writing incorrect queries such as `INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)` when they don't enable case sensitive conf.

### How was this patch tested?

The new added test will fail without this change.

Closes #31669 from zsxwing/SPARK-34556.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-01 13:55:35 +09:00
Kent Yao 1afe284ed8 [SPARK-34570][SQL] Remove dead code from constructors of [Hive]SessionStateBuilder
### What changes were proposed in this pull request?

the parameter - `options` is never used. The changes here was part of https://github.com/apache/spark/pull/30642, It got reverted for easier backporting #30642 as a hotfix by dad24543aa, this PR brings it back to master.

### Why are the changes needed?

remove unless dead code

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

no

### How was this patch tested?

Passing CI is enough.

Closes #31683 from yaooqinn/SPARK-34570.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:30:18 +09:00
Angerszhuuuu d574308864 [SPARK-34579][SQL][TEST] Fix wrong UT in SQLQuerySuite
### What changes were proposed in this pull request?
Some UT in SQLQuerySuite is  not incorrect, it have wrong table name in `withTable`, this pr to make it correct.

### Why are the changes needed?
Fix UT

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

### How was this patch tested?
Existed UT

Closes #31681 from AngersZhuuuu/SPARK-34569.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-02-28 16:21:42 -08:00
Shardul Mahadik 0216051aca [SPARK-34506][CORE] ADD JAR with ivy coordinates should be compatible with Hive transitive behavior
### What changes were proposed in this pull request?
SPARK-33084 added the ability to use ivy coordinates with `SparkContext.addJar`. PR #29966 claims to mimic Hive behavior although I found a few cases where it doesn't

1) The default value of the transitive parameter is false, both in case of parameter not being specified in coordinate or parameter value being invalid. The Hive behavior is that transitive is [true if not specified](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L169)) in the coordinate and [false for invalid values](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L124)). Also, regardless of Hive, I think a default of true for the transitive parameter also matches [ivy's own defaults](https://ant.apache.org/ivy/history/2.5.0/ivyfile/dependency.html#_attributes).

2) The parameter value for transitive parameter is regarded as case-sensitive [based on the understanding](https://github.com/apache/spark/pull/29966#discussion_r547752259) that Hive behavior is case-sensitive. However, this is not correct, Hive [treats the parameter value case-insensitively](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L122)).

I propose that we be compatible with Hive for these behaviors

### Why are the changes needed?
To make `ADD JAR` with ivy coordinates compatible with Hive's transitive behavior

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

The user-facing changes here are within master as the feature introduced in SPARK-33084 has not been released yet
1. Previously an ivy coordinate without `transitive` parameter specified did not resolve transitive dependency, now it does.
2. Previously an `transitive` parameter value was treated case-sensitively. e.g. `transitive=TRUE` would be treated as false as it did not match exactly `true`. Now it will be treated case-insensitively.

### How was this patch tested?

Modified existing unit tests to test new behavior
Add new unit test to cover usage of `exclude` with unspecified `transitive`

Closes #31623 from shardulm94/spark-34506.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:10:20 +09:00
Yuming Wang d07fc3076b [SPARK-33687][SQL] Support analyze all tables in a specific database
### What changes were proposed in this pull request?

This pr add support analyze all tables in a specific database:
```g4
 ANALYZE TABLES ((FROM | IN) multipartIdentifier)? COMPUTE STATISTICS (identifier)?
```

### Why are the changes needed?

1. Make it easy to analyze all tables in a specific database.
2. PostgreSQL has a similar implementation: https://www.postgresql.org/docs/12/sql-analyze.html.

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

No.

### How was this patch tested?

The feature tested by unit test.
The documentation tested by regenerating the documentation:

menu-sql.yaml |  sql-ref-syntax-aux-analyze-tables.md
-- | --
![image](https://user-images.githubusercontent.com/5399861/109098769-dc33a200-775c-11eb-86b1-55531e5425e0.png) | ![image](https://user-images.githubusercontent.com/5399861/109098841-02594200-775d-11eb-8588-de8da97ec94a.png)

Closes #30648 from wangyum/SPARK-33687.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:06:47 +09:00
Yuming Wang 54c053afb0 [SPARK-34479][SQL] Add zstandard codec to Avro compression codec list
### What changes were proposed in this pull request?

Avro add zstandard codec since AVRO-2195. This pr add zstandard codec to Avro compression codec list.

### Why are the changes needed?

To make Avro support zstandard codec.

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

No.

### How was this patch tested?

Unit test.

Closes #31673 from wangyum/SPARK-34479.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-02-27 10:31:42 -08:00
Ruifeng Zheng 05069ff4ce [SPARK-34353][SQL] CollectLimitExec avoid shuffle if input rdd has 0/1 partition
### What changes were proposed in this pull request?
if child rdd has only one partition or zero partition, skip the shuffle

### Why are the changes needed?
skip shuffle if possible

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

### How was this patch tested?
existing testsuites

Closes #31468 from zhengruifeng/collect_limit_single_partition.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-27 16:48:20 +09:00
ShiKai Wang 56e664c717 [SPARK-34392][SQL] Support ZoneOffset +h:mm in DateTimeUtils. getZoneId
### What changes were proposed in this pull request?
To support +8:00 in Spark3 when execute sql
`select to_utc_timestamp("2020-02-07 16:00:00", "GMT+8:00")`

### Why are the changes needed?
+8:00 this format is supported in PostgreSQL,hive, presto, but not supported in Spark3
https://issues.apache.org/jira/browse/SPARK-34392

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

### How was this patch tested?
unit test

Closes #31624 from Karl-WangSK/zone.

Lead-authored-by: ShiKai Wang <wskqing@gmail.com>
Co-authored-by: Karl-WangSK <shikai.wang@linkflowtech.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-26 11:03:20 -06:00
tanel.kiis@gmail.com 67ec4f7f67 [SPARK-33971][SQL] Eliminate distinct from more aggregates
### What changes were proposed in this pull request?

Add more aggregate expressions to `EliminateDistinct` rule.

### Why are the changes needed?

Distinct aggregation can add a significant overhead. It's better to remove distinct whenever possible.

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

No

### How was this patch tested?

UT

Closes #30999 from tanelk/SPARK-33971_eliminate_distinct.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-02-26 21:59:02 +09:00
Max Gekk c1beb16cc8 [SPARK-34554][SQL] Implement the copy() method in ColumnarMap
### What changes were proposed in this pull request?
Implement `ColumnarMap.copy()` by using the `copy()` method of `ColumnarArray`.

### Why are the changes needed?
To eliminate `java.lang.UnsupportedOperationException` while using `ColumnarMap`.

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

### How was this patch tested?
By running new tests in `ColumnarBatchSuite`.

Closes #31663 from MaxGekk/columnar-map-copy.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 21:33:14 +09:00
ulysses-you 82267acfe8 [SPARK-34550][SQL] Skip InSet null value during push filter to Hive metastore
### What changes were proposed in this pull request?

Skip `InSet` null value during push filter to Hive metastore.

### Why are the changes needed?

If `InSet` contains a null value, we should skip it and push other values to metastore. To keep same behavior with `In`.

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

No.

### How was this patch tested?

Add test.

Closes #31659 from ulysses-you/SPARK-34550.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 21:29:14 +09:00
Cheng Su 7d5021f5ee [SPARK-34533][SQL] Eliminate LEFT ANTI join to empty relation in AQE
### What changes were proposed in this pull request?

I discovered from review discussion - https://github.com/apache/spark/pull/31630#discussion_r581774000 , that we can eliminate LEFT ANTI join (with no join condition) to empty relation, if the right side is known to be non-empty. So with AQE, this is doable similar to https://github.com/apache/spark/pull/29484 .

### Why are the changes needed?

This can help eliminate the join operator during logical plan optimization.
Before this PR, [left side physical plan `execute()` will be called](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L192), so if left side is complicated (e.g. contain broadcast exchange operator), then some computation would happen. However after this PR, the join operator will be removed during logical plan, and nothing will be computed from left side. Potentially it can save resource for these kinds of query.

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

No.

### How was this patch tested?

Added unit tests for positive and negative queries in `AdaptiveQueryExecSuite.scala`.

Closes #31641 from c21/left-anti-aqe.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 11:46:27 +00:00
Wenchen Fan 73857cdd87 [SPARK-34524][SQL] Simplify v2 partition commands resolution
### What changes were proposed in this pull request?

This PR simplifies the resolution of v2 partition commands:
1. Add a common trait for v2 partition commands, so that we don't need to match them one by one in the rules.
2. Make partition spec an expression, so that it's easier to resolve them via tree node transformation.
3. Add `TruncatePartition` so that `TruncateTable` doesn't need to be a v2 partition command.
4. Simplify `CheckAnalysis` to only check if the table is partitioned. For partitioned tables, partition spec is always resolved, so we don't need to check it. The `SupportsAtomicPartitionManagement` check is also done in the runtime. Since Spark eagerly executes commands, exception in runtime will also be thrown at analysis time.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31637 from cloud-fan/simplify.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 11:44:42 +00:00
Max Gekk 5c7d019b60 [SPARK-34543][SQL] Respect the spark.sql.caseSensitive config while resolving partition spec in v1 SET LOCATION
### What changes were proposed in this pull request?
Preprocess the partition spec passed to the V1 `ALTER TABLE .. SET LOCATION` implementation `AlterTableSetLocationCommand`, and normalize the passed spec according to the partition columns w.r.t the case sensitivity flag  **spark.sql.caseSensitive**.

### Why are the changes needed?
V1 `ALTER TABLE .. SET LOCATION` is case sensitive in fact, and doesn't respect the SQL config **spark.sql.caseSensitive** which is false by default, for instance:
```sql
spark-sql> CREATE TABLE tbl (id INT, part INT) PARTITIONED BY (part);
spark-sql> INSERT INTO tbl PARTITION (part=0) SELECT 0;
spark-sql> SHOW TABLE EXTENDED LIKE 'tbl' PARTITION (part=0);
Location: file:/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0
spark-sql> ALTER TABLE tbl ADD PARTITION (part=1);
spark-sql> SELECT * FROM tbl;
0	0
```
Create new partition folder in the file system:
```
$ cp -r /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/aaa
```
Set new location for the partition part=1:
```sql
spark-sql> ALTER TABLE tbl PARTITION (part=1) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/aaa';
spark-sql> SELECT * FROM tbl;
0	0
0	1
spark-sql> ALTER TABLE tbl ADD PARTITION (PART=2);
spark-sql> SELECT * FROM tbl;
0	0
0	1
```
Set location for a partition in the upper case:
```
$ cp -r /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/part=0 /Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb
```
```sql
spark-sql> ALTER TABLE tbl PARTITION (PART=2) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb';
Error in query: Partition spec is invalid. The spec (PART) must match the partition spec (part) defined in table '`default`.`tbl`'
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, the command above works as expected:
```sql
spark-sql> ALTER TABLE tbl PARTITION (PART=2) SET LOCATION '/Users/maximgekk/proj/set-location-case-sense/spark-warehouse/tbl/bbb';
spark-sql> SELECT * FROM tbl;
0	0
0	1
0	2
```

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *CatalogedDDLSuite"
```

Closes #31651 from MaxGekk/set-location-case-sense.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-26 15:28:57 +08:00
yangjie01 0d3a9cd3c9 [SPARK-34535][SQL] Cleanup unused symbol in Orc related code
### What changes were proposed in this pull request?
Cleanup unused symbol in Orc related code as follows:

- `OrcDeserializer` : parameter `dataSchema` in constructor
- `OrcFilters`  : parameter `schema ` in method `convertibleFilters`.
- `OrcPartitionReaderFactory`: ignore return value of `OrcUtils.orcResultSchemaString` in  method `buildReader(file: PartitionedFile)`

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

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

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31644 from LuciferYang/cleanup-orc-unused-symbol.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-26 09:20:40 +09:00
Wenchen Fan dffb01f28a [SPARK-34152][SQL][FOLLOWUP] Do not uncache the temp view if it doesn't exist
### What changes were proposed in this pull request?

This PR fixes a mistake in https://github.com/apache/spark/pull/31273. When CREATE OR REPLACE a temp view, we need to uncache the to-be-replaced existing temp view. However, we shouldn't uncache if there is no existing temp view.

This doesn't cause real issues because the uncache action is failure-safe. But it produces a lot of warning messages.

### Why are the changes needed?

Avoid unnecessary warning logs.

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

no

### How was this patch tested?

manually run tests and check the warning messages.

Closes #31650 from cloud-fan/warnning.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-25 15:25:41 -08:00
Liang-Chi Hsieh f7ac2d655c [SPARK-34474][SQL] Remove unnecessary Union under Distinct/Deduplicate
### What changes were proposed in this pull request?

This patch proposes to let optimizer to remove unnecessary `Union` under `Distinct`/`Deduplicate`.

### Why are the changes needed?

For an `Union` under `Distinct`/`Deduplicate`, if its children are all the same, we can just keep one among them and remove the `Union`.

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

No

### How was this patch tested?

Unit tests.

Closes #31595 from viirya/remove-union.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-25 12:41:07 -08:00
Yuming Wang 4a3200b08a [SPARK-34436][SQL] DPP support LIKE ANY/ALL expression
### What changes were proposed in this pull request?

This pr make DPP support LIKE ANY/ALL expression:
```sql
SELECT date_id, product_id FROM fact_sk f
JOIN dim_store s
ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%')
```

### 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 #31563 from wangyum/SPARK-34436.

Lead-authored-by: Yuming Wang <yumwang@apache.org>
Co-authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-25 18:07:39 +08:00
Max Gekk c56af69cdf [SPARK-34518][SQL] Rename AlterTableRecoverPartitionsCommand to RepairTableCommand
### What changes were proposed in this pull request?
Rename the execution node `AlterTableRecoverPartitionsCommand` for the commands:
- `MSCK REPAIR TABLE table [{ADD|DROP|SYNC} PARTITIONS]`
- `ALTER TABLE table RECOVER PARTITIONS`

to `RepairTableCommand`.

### Why are the changes needed?
1. After the PR https://github.com/apache/spark/pull/31499, `ALTER TABLE table RECOVER PARTITIONS` is equal to `MSCK REPAIR TABLE table ADD PARTITIONS`. And mapping of the generic command `MSCK REPAIR TABLE` to the more specific execution node `AlterTableRecoverPartitionsCommand` can confuse devs in the future.
2. `ALTER TABLE table RECOVER PARTITIONS` does not support any options/extensions. So, additional parameters `enableAddPartitions` and `enableDropPartitions` in `AlterTableRecoverPartitionsCommand` confuse as well.

### Does this PR introduce _any_ user-facing change?
No because this is internal API.

### How was this patch tested?
By running the existing test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
$ build/sbt "test:testOnly *AlterTableRecoverPartitionsParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableSuite"
$ build/sbt "test:testOnly *MsckRepairTableParserSuite"
```

Closes #31635 from MaxGekk/rename-recover-partitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-25 09:32:41 +00:00
Gabor Somogyi 44eadb943b [SPARK-34497][SQL] Fix built-in JDBC connection providers to restore JVM security context changes
### What changes were proposed in this pull request?
Some of the built-in JDBC connection providers are changing the JVM security context to do the authentication which is fine. The problematic part is that executors can be reused by another query. The following situation leads to incorrect behaviour:
* Query1 opens JDBC connection and changes JVM security context in Executor1
* Query2 tries to open JDBC connection but it realizes there is already an entry for that DB type in Executor1
* Query2 is not changing JVM security context and uses Query1 keytab and principal
* Query2 fails with authentication error

In this PR I've changed to code such a way that JVM security context is changed all the time but only temporarily until the connection built-up and then rolled back. Since `getConnection` is synchronised with `SecurityConfigurationLock` it ends-up in correct behaviour without any race.

### Why are the changes needed?
Incorrect JVM security context handling.

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

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

Closes #31622 from gaborgsomogyi/SPARK-34497.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-25 09:25:17 +09:00
ulysses-you 999d3b89b6 [SPARK-34515][SQL] Fix NPE if InSet contains null value during getPartitionsByFilter
### What changes were proposed in this pull request?

Skip null value during rewrite `InSet` to `>= and <=` at getPartitionsByFilter.

### Why are the changes needed?

Spark will convert `InSet` to `>= and <=` if it's values size over `spark.sql.hive.metastorePartitionPruningInSetThreshold` during pruning partition . At this case, if values contain a null, we will get such exception 
 
```
java.lang.NullPointerException
 at org.apache.spark.unsafe.types.UTF8String.compareTo(UTF8String.java:1389)
 at org.apache.spark.unsafe.types.UTF8String.compareTo(UTF8String.java:50)
 at scala.math.LowPriorityOrderingImplicits$$anon$3.compare(Ordering.scala:153)
 at java.util.TimSort.countRunAndMakeAscending(TimSort.java:355)
 at java.util.TimSort.sort(TimSort.java:220)
 at java.util.Arrays.sort(Arrays.java:1438)
 at scala.collection.SeqLike.sorted(SeqLike.scala:659)
 at scala.collection.SeqLike.sorted$(SeqLike.scala:647)
 at scala.collection.AbstractSeq.sorted(Seq.scala:45)
 at org.apache.spark.sql.hive.client.Shim_v0_13.convert$1(HiveShim.scala:772)
 at org.apache.spark.sql.hive.client.Shim_v0_13.$anonfun$convertFilters$4(HiveShim.scala:826)
 at scala.collection.immutable.Stream.flatMap(Stream.scala:489)
 at org.apache.spark.sql.hive.client.Shim_v0_13.convertFilters(HiveShim.scala:826)
 at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:848)
 at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:750)
```

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

Yes, bug fix.

### How was this patch tested?

Add test.

Closes #31632 from ulysses-you/SPARK-34515.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 21:32:19 +08:00
Cheng Su 6ef57d31cd [SPARK-34514][SQL] Push down limit for LEFT SEMI and LEFT ANTI join
### What changes were proposed in this pull request?

I found out during code review of https://github.com/apache/spark/pull/31567#discussion_r577379572, where we can push down limit to the left side of LEFT SEMI and LEFT ANTI join, if the join condition is empty.

Why it's safe to push down limit:

The semantics of LEFT SEMI join without condition:
(1). if right side is non-empty, output all rows from left side.
(2). if right side is empty, output nothing.

The semantics of LEFT ANTI join without condition:
(1). if right side is non-empty, output nothing.
(2). if right side is empty, output all rows from left side.

With the semantics of output all rows from left side or nothing (all or nothing), it's safe to push down limit to left side.
NOTE: LEFT SEMI / LEFT ANTI join with non-empty condition is not safe for limit push down, because output can be a portion of left side rows.

Reference: physical operator implementation for LEFT SEMI / LEFT ANTI join without condition - https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala#L200-L204 .

### Why are the changes needed?

Better performance. Save CPU and IO for these joins, as limit being pushed down before join.

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

No.

### How was this patch tested?

Added unit test in `LimitPushdownSuite.scala` and `SQLQuerySuite.scala`.

Closes #31630 from c21/limit-pushdown.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 10:23:01 +00:00
beliefer 14934f42d0 [SPARK-33599][SQL][FOLLOWUP] Group exception messages in catalyst/analysis
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/30717
Maybe some contributors don't know the job and added some exception by the old way.

### Why are the changes needed?
It will largely help with standardization of error messages and its maintenance.

### Does this PR introduce _any_ user-facing change?
No. Error messages remain unchanged.

### How was this patch tested?
No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31316 from beliefer/SPARK-33599-followup.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 07:28:44 +00:00
Terry Kim 714ff73d4a [SPARK-34152][SQL] Make CreateViewStatement.child to be LogicalPlan's children so that it's resolved in analyze phase
### What changes were proposed in this pull request?

This PR proposes to make `CreateViewStatement.child` to be `LogicalPlan`'s `children` so that it's resolved in the analyze phase.

### Why are the changes needed?

Currently, the `CreateViewStatement.child` is resolved when the create view command runs, which is inconsistent with other plan resolutions. For example, you may see the following in the physical plan:
```
== Physical Plan ==
Execute CreateViewCommand (1)
   +- CreateViewCommand (2)
         +- Project (4)
            +- UnresolvedRelation (3)
```

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

Yes. For the example, you will now see the resolved plan:
```
== Physical Plan ==
Execute CreateViewCommand (1)
   +- CreateViewCommand (2)
         +- Project (5)
            +- SubqueryAlias (4)
               +- LogicalRelation (3)
```

### How was this patch tested?

Updated existing tests.

Closes #31273 from imback82/spark-34152.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 06:50:11 +00:00
Gengliang Wang 5d9cfd727c [SPARK-34246][SQL] New type coercion syntax rules in ANSI mode
### What changes were proposed in this pull request?

In Spark ANSI mode, the type coercion rules are based on the type precedence lists of the input data types.
As per the section "Type precedence list determination" of "ISO/IEC 9075-2:2011
Information technology — Database languages - SQL — Part 2: Foundation (SQL/Foundation)", the type precedence lists of primitive data types are as following:

- Byte: Byte, Short, Int, Long, Decimal, Float, Double
- Short: Short, Int, Long, Decimal, Float, Double
- Int: Int, Long, Decimal, Float, Double
- Long: Long, Decimal, Float, Double
- Decimal: Any wider Numeric type
- Float: Float, Double
- Double: Double
- String: String
- Date: Date, Timestamp
- Timestamp: Timestamp
- Binary: Binary
- Boolean: Boolean
- Interval: Interval

As for complex data types, Spark will determine the precedent list recursively based on their sub-types.

With the definition of type precedent list, the general type coercion rules are as following:
- Data type S is allowed to be implicitly cast as type T iff T is in the precedence list of S
- Comparison is allowed iff the data type precedence list of both sides has at least one common element. When evaluating the comparison, Spark casts both sides as the tightest common data type of their precedent lists.
- There should be at least one common data type among all the children's precedence lists for the following operators. The data type of the operator is the tightest common precedent data type.
```
 In, Except(odd), Intersect, Greatest, Least, Union, If, CaseWhen, CreateArray, Array Concat,Sequence, MapConcat, CreateMap
```

- For complex types (struct, array, map), Spark recursively looks into the element type and applies the rules above. If the element nullability is converted from true to false, add runtime null check to the elements.

Note: this new type coercion system will allow implicit converting String type literals as other primitive types, in case of breaking too many existing Spark SQL queries. This is a special rule and it is not from the ANSI SQL standard.
### Why are the changes needed?

The current type coercion rules are complex. Also, they are very hard to describe and understand. For details please refer the attached documentation "Default Type coercion rules of Spark"
[Default Type coercion rules of Spark.pdf](https://github.com/apache/spark/files/5874362/Default.Type.coercion.rules.of.Spark.pdf)

This PR is to create a new and strict type coercion system under ANSI mode. The rules are simple and clean, so that users can follow them easily

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

Yes,  new implicit cast syntax rules in ANSI mode. All the details are in the first section of this description.

### How was this patch tested?

Unit tests

Closes #31349 from gengliangwang/ansiImplicitConversion.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-02-24 13:40:58 +08:00
Max Gekk f64fc22466 [SPARK-34290][SQL] Support v2 TRUNCATE TABLE
### What changes were proposed in this pull request?
Implement the v2 execution node for the `TRUNCATE TABLE` command.

### Why are the changes needed?
To have feature parity with DS v1, and support truncation of v2 tables.

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

### How was this patch tested?
By running the unified tests for v1 and v2 tables:
```
$ build/sbt -Phive -Phive-thriftserver "test:testOnly *TruncateTableSuite"
```

Closes #31605 from MaxGekk/truncate-table-v2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 05:21:11 +00:00
HyukjinKwon 80bad086c8 Revert "[SPARK-32703][SQL] Replace deprecated API calls from SpecificParquetRecordReaderBase"
This reverts commit 27873280ff.
2021-02-24 11:36:54 +09:00
Yuming Wang b5afff59fa [SPARK-26138][SQL] Pushdown limit through InnerLike when condition is empty
### What changes were proposed in this pull request?

This pr pushdown limit through InnerLike when condition is empty(Origin pr: #23104). For example:
```sql
CREATE TABLE t1 using parquet AS SELECT id AS a, id AS b FROM range(2);
CREATE TABLE t2 using parquet AS SELECT id AS d FROM range(2);
SELECT * FROM t1 CROSS JOIN t2 LIMIT 10;
```
Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- CollectLimit 10
   +- BroadcastNestedLoopJoin BuildRight, Cross
      :- FileScan parquet default.t1[a#5L,b#6L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/tg/f5mz46090wg7swzgdc69f8q03965_0/T/warehous..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
      +- BroadcastExchange IdentityBroadcastMode, [id=#43]
         +- FileScan parquet default.t2[d#7L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/tg/f5mz46090wg7swzgdc69f8q03965_0/T/warehous..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<d:bigint>
```
After this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- CollectLimit 10
   +- BroadcastNestedLoopJoin BuildRight, Cross
      :- LocalLimit 10
      :  +- FileScan parquet default.t1[a#5L,b#6L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/tg/f5mz46090wg7swzgdc69f8q03965_0/T/warehous..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<a:bigint,b:bigint>
      +- BroadcastExchange IdentityBroadcastMode, [id=#51]
         +- LocalLimit 10
            +- FileScan parquet default.t2[d#7L] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/tg/f5mz46090wg7swzgdc69f8q03965_0/T/warehous..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<d:bigint>
```

### 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 #31567 from wangyum/SPARK-26138.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-02-24 09:50:13 +08:00
Max Gekk 7f27d33a3c [SPARK-31891][SQL] Support MSCK REPAIR TABLE .. [{ADD|DROP|SYNC} PARTITIONS]
### What changes were proposed in this pull request?

In the PR, I propose to extend the `MSCK REPAIR TABLE` command, and support new options `{ADD|DROP|SYNC} PARTITIONS`. In particular:

1. Extend the logical node `RepairTable`, and add two new flags `enableAddPartitions` and `enableDropPartitions`.
2. Add similar flags to the v1 execution node `AlterTableRecoverPartitionsCommand`
3. Add new method `dropPartitions()` to `AlterTableRecoverPartitionsCommand` which drops partitions from the catalog if their locations in the file system don't exist.
4. Updated public docs about the `MSCK REPAIR TABLE` command:
<img width="1037" alt="Screenshot 2021-02-16 at 13 46 39" src="https://user-images.githubusercontent.com/1580697/108052607-7446d280-705d-11eb-8e25-7398254787a4.png">

Closes #31097

### Why are the changes needed?
- The changes allow to recover tables with removed partitions. The example below portraits the problem:
```sql
spark-sql> create table tbl2 (col int, part int) partitioned by (part);
spark-sql> insert into tbl2 partition (part=1) select 1;
spark-sql> insert into tbl2 partition (part=0) select 0;
spark-sql> show table extended like 'tbl2' partition (part = 0);
default	tbl2	false	Partition Values: [part=0]
Location: file:/Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
...
```
Remove the partition (part = 0) from the filesystem:
```
$ rm -rf /Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
```
Even after recovering, we cannot query the table:
```sql
spark-sql> msck repair table tbl2;
spark-sql> select * from tbl2;
21/01/08 22:49:13 ERROR SparkSQLDriver: Failed in [select * from tbl2]
org.apache.hadoop.mapred.InvalidInputException: Input path does not exist: file:/Users/maximgekk/proj/apache-spark/spark-warehouse/tbl2/part=0
```

- To have feature parity with Hive: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-RecoverPartitions(MSCKREPAIRTABLE)

### Does this PR introduce _any_ user-facing change?
Yes. After the changes, we can query recovered table:
```sql
spark-sql> msck repair table tbl2 sync partitions;
spark-sql> select * from tbl2;
1	1
spark-sql> show partitions tbl2;
part=1
```

### How was this patch tested?
- By running the modified test suite:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableParserSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *PlanResolutionSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsSuite"
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRecoverPartitionsParallelSuite"
```
- Added unified v1 and v2 tests for `MSCK REPAIR TABLE`:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *MsckRepairTableSuite"
```

Closes #31499 from MaxGekk/repair-table-drop-partitions.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-23 13:45:15 -08:00
Wenchen Fan 95e45c6257 [SPARK-34168][SQL][FOLLOWUP] Improve DynamicPartitionPruningSuiteBase
### What changes were proposed in this pull request?

A few minor improvements for `DynamicPartitionPruningSuiteBase`.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31625 from cloud-fan/followup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-23 13:41:24 -08:00
Wenchen Fan 0d5d248bdc [SPARK-34508][SQL][TEST] Skip HiveExternalCatalogVersionsSuite if network is down
### What changes were proposed in this pull request?

It's possible that the network is down when running Spark tests, and it's annoying to see `HiveExternalCatalogVersionsSuite` keep failing.

This PR proposes to skip this test suite if we can't get the latest Spark version from the Apache website.

### Why are the changes needed?

Make the Spark tests more robust.

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

no

### How was this patch tested?

N/A

Closes #31627 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-23 13:35:29 -08:00
Huaxin Gao 443139b601 [SPARK-34502][SQL] Remove unused parameters in join methods
### What changes were proposed in this pull request?

Remove unused parameters in `CoalesceBucketsInJoin`, `UnsafeCartesianRDD` and `ShuffledHashJoinExec`.

### Why are the changes needed?
Clean up

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

### How was this patch tested?
Existing tests

Closes #31617 from huaxingao/join-minor.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-02-23 12:18:43 -08:00
Wenchen Fan 429f8af9b6 Revert "[SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command"
This reverts commit 9a566f83a0.
2021-02-24 02:38:22 +08:00