Commit graph

11673 commits

Author SHA1 Message Date
Gengliang Wang 99eb3ff226 [SPARK-36227][SQL][3.2] Remove TimestampNTZ type support in Spark 3.2
### What changes were proposed in this pull request?

Remove TimestampNTZ type support in the production code of Spark 3.2.
To archive the goal, this PR adds the check "Utils.isTesting" in the following code branches:
- keyword "timestamp_ntz" and "timestamp_ltz" in parser
- New expressions from https://issues.apache.org/jira/browse/SPARK-35662
- Using java.time.localDateTime as the external type for TimestampNTZType
- `SQLConf.timestampType` which determines the default timestamp type of Spark SQL.

This is to minimize the code difference between the master branch. So that future users won't think TimestampNTZ is already available in Spark 3.2.
The downside is that users can still find TimestampNTZType under package `org.apache.spark.sql.types`. There should be nothing left other than this.
### Why are the changes needed?

As of now, there are some blockers for delivering the TimestampNTZ project in Spark 3.2:

- In the Hive Thrift server, both TimestampType and TimestampNTZType are mapped to the same timestamp type, which can cause confusion for users.
- For the Parquet data source, the new written TimestampNTZType Parquet columns will be read as TimestampType in old Spark releases. Also, we need to decide the merge schema for files mixed with TimestampType and TimestampNTZ type.
- The type coercion rules for TimestampNTZType are incomplete. For example, what should the data type of the in clause "IN(Timestamp'2020-01-01 00:00:00', TimestampNtz'2020-01-01 00:00:00') be.
- It is tricky to support TimestampNTZType in JSON/CSV data readers. We need to avoid regressions as possible as we can.

There are 10 days left for the expected 3.2 RC date. So, I propose to **release the TimestampNTZ type in Spark 3.3 instead of Spark 3.2**. So that we have enough time to make considerate designs for the issues.

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

No.

### How was this patch tested?

Existing Unit tests + manual tests from spark-shell to validate the changes are gone.
New functions
```
spark.sql("select to_timestamp_ntz'2021-01-01 00:00:00'").show()
spark.sql("select to_timestamp_ltz'2021-01-01 00:00:00'").show()
spark.sql("select make_timestamp_ntz(1,1,1,1,1,1)").show()
spark.sql("select make_timestamp_ltz(1,1,1,1,1,1)").show()
spark.sql("select localtimestamp()").show()
```
The SQL configuration `spark.sql.timestampType` should not work in 3.2
```
spark.conf.set("spark.sql.timestampType", "TIMESTAMP_NTZ")
spark.sql("select make_timestamp(1,1,1,1,1,1)").schema
spark.sql("select to_timestamp('2021-01-01 00:00:00')").schema
spark.sql("select timestamp'2021-01-01 00:00:00'").schema
Seq((1, java.sql.Timestamp.valueOf("2021-01-01 00:00:00"))).toDF("i", "ts").write.partitionBy("ts").parquet("/tmp/test")
spark.read.parquet("/tmp/test").schema
```
LocalDateTime is not supported as a built-in external type:
```
Seq(LocalDateTime.now()).toDF()
org.apache.spark.sql.catalyst.expressions.Literal(java.time.LocalDateTime.now())
org.apache.spark.sql.catalyst.expressions.Literal(0L, TimestampNTZType)
```

Closes #33444 from gengliangwang/banNTZ.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-21 09:55:09 -07:00
Kent Yao 7d363733ac [SPARK-36213][SQL] Normalize PartitionSpec for Describe Table Command with PartitionSpec
### What changes were proposed in this pull request?

This fixes a case sensitivity issue for desc table commands with partition specified.

### Why are the changes needed?

bugfix

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

yes, but it's a bugfix

### How was this patch tested?

new tests

#### before
```
+-- !query
+DESC EXTENDED t PARTITION (C='Us', D=1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+Partition spec is invalid. The spec (C, D) must match the partition spec (c, d) defined in table '`default`.`t`'
+
```

#### after

https://github.com/apache/spark/pull/33424/files#diff-554189c49950974a948f99fa9b7436f615052511660c6a0ae3062fa8ca0a327cR328

Closes #33424 from yaooqinn/SPARK-36213.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit 4cd6cfc773)
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-22 00:53:12 +08:00
Shardul Mahadik 1ce678b2aa [SPARK-28266][SQL] convertToLogicalRelation should not interpret path property when reading Hive tables
### What changes were proposed in this pull request?

For non-datasource Hive tables, e.g. tables written outside of Spark (through Hive or Trino), we have certain optimzations in Spark where we use Spark ORC and Parquet datasources to read these tables ([Ref](fbf53dee37/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala (L128))) rather than using the Hive serde.
If such a table contains a `path` property, Spark will try to list this path property in addition to the table location when creating an `InMemoryFileIndex`. ([Ref](fbf53dee37/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L575))) This can lead to wrong data if `path` property points to a directory location or an error if `path` is not a location. A concrete example is provided in [SPARK-28266 (comment)](https://issues.apache.org/jira/browse/SPARK-28266?focusedCommentId=17380170&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17380170).

Since these tables were not written through Spark, Spark should not interpret this `path` property as it can be set by an external system with a different meaning.

### Why are the changes needed?

For better compatibility with Hive tables generated by other platforms (non-Spark)

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

No

### How was this patch tested?

Added unit test

Closes #33328 from shardulm94/spark-28266.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 685c3fd05b)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-21 22:40:59 +08:00
Wenchen Fan f4291e373e [SPARK-36228][SQL] Skip splitting a skewed partition when some map outputs are removed
### What changes were proposed in this pull request?

Sometimes, AQE skew join optimization can fail with NPE. This is because AQE tries to get the shuffle block sizes, but some map outputs are missing due to the executor lost or something.

This PR fixes this bug by skipping skew join handling if some map outputs are missing in the `MapOutputTracker`.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

a new UT

Closes #33445 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 9c8a3d3975)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-21 22:18:14 +08:00
Wenchen Fan b5c0f6c774 [SPARK-36020][SQL][FOLLOWUP] RemoveRedundantProjects should retain the LOGICAL_PLAN_TAG tag
### What changes were proposed in this pull request?

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

https://github.com/apache/spark/pull/33222 made a mistake that, `RemoveRedundantProjects` may lose the `LOGICAL_PLAN_TAG` tag, even though the logical plan link is retained. This was actually caught by the test `LogicalPlanTagInSparkPlanSuite`, but was not being taken care of.

There is no problem so far, but losing information can always lead to potential bugs.

### Why are the changes needed?

fix a mistake

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

no

### How was this patch tested?

existing test

Closes #33442 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 94aece4325)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-21 14:03:22 +08:00
Rahul Mahadev 0d60cb51c0 [SPARK-36132][SS][SQL] Support initial state for batch mode of flatMapGroupsWithState
### What changes were proposed in this pull request?
Adding support for accepting an initial state with flatMapGroupsWithState in batch mode.

### Why are the changes needed?
SPARK-35897  added support for accepting an initial state for streaming queries using flatMapGroupsWithState. the code flow is separate for batch and streaming and required a different PR.

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

Yes as discussed above flatMapGroupsWithState in batch mode can accept an initialState, previously this would throw an UnsupportedOperationException

### How was this patch tested?

Added relevant unit tests in FlatMapGroupsWithStateSuite and modified the  tests `JavaDatasetSuite`

Closes #33336 from rahulsmahadev/flatMapGroupsWithStateBatch.

Authored-by: Rahul Mahadev <rahul.mahadev@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
(cherry picked from commit efcce23b91)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2021-07-21 01:51:01 -04:00
Liang-Chi Hsieh 0b14ab12a2 [SPARK-36030][SQL][FOLLOW-UP][3.2] Remove duplicated test suiteRemove duplicated test suite
### What changes were proposed in this pull request?

Removes `FileFormatDataWriterMetricSuite` which duplicated.

### Why are the changes needed?

`FileFormatDataWriterMetricSuite` should be renamed to `InMemoryTableMetricSuite`. But it was wrongly copied.

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

No

### How was this patch tested?

Existing tests.

Closes #33454 from viirya/SPARK-36030-followup-3.2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-20 22:29:57 -07:00
Hyukjin Kwon 6041d1c51b [SPARK-36030][SQL][FOLLOW-UP] Avoid procedure syntax deprecated in Scala 2.13
### What changes were proposed in this pull request?

This PR avoid using procedure syntax deprecated in Scala 2.13.

https://github.com/apache/spark/runs/3120481756?check_suite_focus=true

```
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriterMetricSuite.scala:44:90: procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testMetricOnDSv2`'s return type
[error]   private def testMetricOnDSv2(func: String => Unit, checker: Map[Long, String] => Unit) {
[error]                                                                                          ^
[error] /home/runner/work/spark/spark/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala:44:90: procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testMetricOnDSv2`'s return type
[error]   private def testMetricOnDSv2(func: String => Unit, checker: Map[Long, String] => Unit) {
[error]                                                                                          ^
[warn] 100 warnings found
[error] two errors found
[error] (sql / Test / compileIncremental) Compilation failed
[error] Total time: 579 s (09:39), completed Jul 21, 2021 4:14:26 AM
```

### Why are the changes needed?

To make the build compatible with Scala 2.13 in Spark.

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

No, dev-only.

### How was this patch tested?

Manually tested:

```bash
./dev/change-scala-version.sh 2.13
./build/mvn -DskipTests -Phive-2.3 -Phive clean package -Pscala-2.13
```

Closes #33452 from HyukjinKwon/SPARK-36030.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 99006e515b)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-21 14:09:35 +09:00
Liang-Chi Hsieh 86d1fb4698 [SPARK-36030][SQL] Support DS v2 metrics at writing path
### What changes were proposed in this pull request?

We add the interface for DS v2 metrics in SPARK-34366. It is only added for reading path, though. This patch extends the metrics interface to writing path.

### Why are the changes needed?

Complete DS v2 metrics interface support in writing path.

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

No. For developer, yes, as this adds metrics support at DS v2 writing path.

### How was this patch tested?

Added test.

Closes #33239 from viirya/v2-write-metrics.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 2653201b0a)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-20 20:20:48 -07:00
gengjiaan 9a7c59c99c [SPARK-36222][SQL] Step by days in the Sequence expression for dates
### What changes were proposed in this pull request?
The current implement of `Sequence` expression not support step by days for dates.
```
spark-sql> select sequence(date'2021-07-01', date'2021-07-10', interval '3' day);
Error in query: cannot resolve 'sequence(DATE '2021-07-01', DATE '2021-07-10', INTERVAL '3' DAY)' due to data type mismatch:
sequence uses the wrong parameter type. The parameter type must conform to:
1. The start and stop expressions must resolve to the same type.
2. If start and stop expressions resolve to the 'date' or 'timestamp' type
then the step expression must resolve to the 'interval' or
'interval year to month' or 'interval day to second' type,
otherwise to the same type as the start and stop expressions.
         ; line 1 pos 7;
'Project [unresolvedalias(sequence(2021-07-01, 2021-07-10, Some(INTERVAL '3' DAY), Some(Europe/Moscow)), None)]
+- OneRowRelation
```

### Why are the changes needed?
`DayTimeInterval` has day granularity should as step for dates.

### Does this PR introduce _any_ user-facing change?
'Yes'.
Sequence expression will supports step by `DayTimeInterval` has day granularity for dates.

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

Closes #33439 from beliefer/SPARK-36222.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit c0d84e6cf1)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-20 19:17:09 +03:00
Koert Kuipers a864388b5a [SPARK-36210][SQL] Preserve column insertion order in Dataset.withColumns
### What changes were proposed in this pull request?
Preserve the insertion order of columns in Dataset.withColumns

### Why are the changes needed?
It is the expected behavior. We preserve insertion order in all other places.

### Does this PR introduce _any_ user-facing change?
No. Currently Dataset.withColumns is not actually used anywhere to insert more than one column. This change is to make sure it behaves as expected when it is used for that purpose in future.

### How was this patch tested?
Added test in DatasetSuite

Closes #33423 from koertkuipers/feat-withcolumns-preserve-order.

Authored-by: Koert Kuipers <koert@tresata.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit bf680bf25a)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-20 09:09:34 -07:00
Karen Feng f55f8820fc [SPARK-36079][SQL] Null-based filter estimate should always be in the range [0, 1]
### What changes were proposed in this pull request?

Forces the selectivity estimate for null-based filters to be in the range `[0,1]`.

### Why are the changes needed?

I noticed in a few TPC-DS query tests that the column statistic null count can be higher than the table statistic row count. In the current implementation, the selectivity estimate for `IsNotNull` is negative.

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

No

### How was this patch tested?

Unit test

Closes #33286 from karenfeng/bound-selectivity-est.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit ddc61e62b9)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-20 21:32:30 +08:00
gengjiaan 0f6cf8abe3 [SPARK-36046][SQL][FOLLOWUP] Implement prettyName for MakeTimestampNTZ and MakeTimestampLTZ
### What changes were proposed in this pull request?
This PR follows https://github.com/apache/spark/pull/33299 and implement `prettyName` for `MakeTimestampNTZ` and `MakeTimestampLTZ` based on the discussion show below
https://github.com/apache/spark/pull/33299/files#r668423810

### Why are the changes needed?
This PR fix the incorrect alias usecase.

### Does this PR introduce _any_ user-facing change?
'No'.
Modifications are transparent to users.

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

Closes #33430 from beliefer/SPARK-36046-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 033a5731b4)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-20 21:31:34 +08:00
Angerszhuuuu 7cd89efca5 [SPARK-36201][SQL][FOLLOWUP] Schema check should check inner field too
### What changes were proposed in this pull request?
When inner field have wrong schema filed name should check field name too.
![image](https://user-images.githubusercontent.com/46485123/126101009-c192d87f-1e18-4355-ad53-1419dacdeb76.png)

### Why are the changes needed?
Early check early faield

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

### How was this patch tested?
Added UT

Closes #33409 from AngersZhuuuu/SPARK-36201.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 251885772d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-20 21:08:36 +08:00
ulysses-you 677104f495 [SPARK-36221][SQL] Make sure CustomShuffleReaderExec has at least one partition
### What changes were proposed in this pull request?

* Add non-empty partition check in `CustomShuffleReaderExec`
* Make sure `OptimizeLocalShuffleReader` doesn't return empty partition

### Why are the changes needed?

Since SPARK-32083, AQE coalesce always return at least one partition, it should be robust to add non-empty check in `CustomShuffleReaderExec`.

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

No

### How was this patch tested?

not need

Closes #33431 from ulysses-you/non-empty-partition.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit b70c25881c)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-20 20:48:51 +08:00
Kent Yao 782dc9a795 [SPARK-36179][SQL] Support TimestampNTZType in SparkGetColumnsOperation
### What changes were proposed in this pull request?

Support TimestampNTZType in SparkGetColumnsOperation

### Why are the changes needed?

TimestampNTZType coverage

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

yes, jdbc end-users will be aware of TimestampNTZType

### How was this patch tested?

add new test

Closes #33393 from yaooqinn/SPARK-36179.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 0c76fb9c01)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-20 09:49:25 +09:00
gengjiaan ab4c160880 [SPARK-36091][SQL] Support TimestampNTZ type in expression TimeWindow
### What changes were proposed in this pull request?
The current implement of `TimeWindow` only supports `TimestampType`. Spark added a new type `TimestampNTZType`, so we should support `TimestampNTZType` in expression `TimeWindow`.

### Why are the changes needed?
 `TimestampNTZType` similar to `TimestampType`, we should support `TimestampNTZType` in expression `TimeWindow`.

### Does this PR introduce _any_ user-facing change?
'Yes'.
`TimeWindow` will accepts `TimestampNTZType`.

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

Closes #33341 from beliefer/SPARK-36091.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 7aa01798c5)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-19 19:24:01 +08:00
Angerszhuuuu 84a6fa22b3 [SPARK-36093][SQL] RemoveRedundantAliases should not change Command's parameter's expression's name
### What changes were proposed in this pull request?
RemoveRedundantAliases may change DataWritingCommand's parameter's attribute name.
In the UT's case before RemoveRedundantAliases the partitionColumns is `CAL_DT`, and change by RemoveRedundantAliases and change to `cal_dt` then case the error case

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
For below SQL case
```
sql("create table t1(cal_dt date) using parquet")
sql("insert into t1 values (date'2021-06-27'),(date'2021-06-28'),(date'2021-06-29'),(date'2021-06-30')")
sql("create view t1_v as select * from t1")
sql("CREATE TABLE t2 USING PARQUET PARTITIONED BY (CAL_DT) AS SELECT 1 AS FLAG,CAL_DT FROM t1_v WHERE CAL_DT BETWEEN '2021-06-27' AND '2021-06-28'")
sql("INSERT INTO t2 SELECT 2 AS FLAG,CAL_DT FROM t1_v WHERE CAL_DT BETWEEN '2021-06-29' AND '2021-06-30'")
```

Before this pr
```
sql("SELECT * FROM t2 WHERE CAL_DT BETWEEN '2021-06-29' AND '2021-06-30'").show
+----+------+
|FLAG|CAL_DT|
+----+------+
+----+------+
sql("SELECT * FROM t2 ").show
+----+----------+
|FLAG|    CAL_DT|
+----+----------+
|   1|2021-06-27|
|   1|2021-06-28|
+----+----------+
```

After this pr
```
sql("SELECT * FROM t2 WHERE CAL_DT BETWEEN '2021-06-29' AND '2021-06-30'").show
+----+------+
|FLAG|CAL_DT|
+----+------+
|   2|2021-06-29|
|   2|2021-06-30|
+----+------+
sql("SELECT * FROM t2 ").show
+----+----------+
|FLAG|    CAL_DT|
+----+----------+
|   1|2021-06-27|
|   1|2021-06-28|
|   2|2021-06-29|
|   2|2021-06-30|
+----+----------+
```

### How was this patch tested?
Added UT

Closes #33324 from AngersZhuuuu/SPARK-36093.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 313f3c5460)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-19 16:22:47 +08:00
Kent Yao d0c4d224e0 [SPARK-36197][SQL] Use PartitionDesc instead of TableDesc for reading hive partitioned tables
### What changes were proposed in this pull request?

A hive partition can have different `PartitionDesc`s from `TableDesc` for describing Serde/InputFormatClass/OutputFormatClass, for a hive partitioned table, we shall respect those in `PartitionDesc`.

### Why are the changes needed?

in many cases, that Spark reads hive tables could result in surprise because of this issue.

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

yes, hive partition table that contains different serde/input/output could be recognized by Spark

### How was this patch tested?

new test added

Closes #33406 from yaooqinn/SPARK-36197.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit ef80356614)
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-19 16:00:12 +08:00
Wenchen Fan 5b98ec2527 [SPARK-36184][SQL] Use ValidateRequirements instead of EnsureRequirements to skip AQE rules that adds extra shuffles
### What changes were proposed in this pull request?

Currently, two AQE rules `OptimizeLocalShuffleReader` and `OptimizeSkewedJoin` run `EnsureRequirements` at the end to check if there are extra shuffles in the optimized plan and revert the optimization if extra shuffles are introduced.

This PR proposes to run `ValidateRequirements` instead, which is much simpler than `EnsureRequirements`. This PR also moves this check to `AdaptiveSparkPlanExec`, so that it's centralized instead of in each rule. After centralization, the batch name of optimizing the final stage is the same as normal stages, which makes more sense.

### Why are the changes needed?

`EnsureRequirements` is a big rule and even contains optimizations (remove unnecessary shuffles). `ValidateRequirements` is much faster to run and can avoid potential bugs as it has no optimization and is a pure check.

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

no

### How was this patch tested?

existing tests.

Closes #33396 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 8396a70ddc)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-19 14:14:58 +08:00
gengjiaan 85f70a1181 [SPARK-36090][SQL] Support TimestampNTZType in expression Sequence
### What changes were proposed in this pull request?
The current implement of `Sequence` accept `TimestampType`, `DateType` and `IntegralType`. This PR will let `Sequence` accepts `TimestampNTZType`.

### Why are the changes needed?
We can generate sequence for timestamp without time zone.

### Does this PR introduce _any_ user-facing change?
'Yes'.
This PR will let `Sequence` accepts `TimestampNTZType`.

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

Closes #33360 from beliefer/SPARK-36090.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 42275bb20d)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-18 20:46:37 +03:00
Kousuke Saruta f7ed6fc6c6 [SPARK-36170][SQL] Change quoted interval literal (interval constructor) to be converted to ANSI interval types
### What changes were proposed in this pull request?

This PR changes the behavior of the quoted interval literals like `SELECT INTERVAL '1 year 2 month'` to be converted to ANSI interval types.

### Why are the changes needed?

The tnit-to-unit interval literals and the unit list interval literals are converted to ANSI interval types but quoted interval literals are still converted to CalendarIntervalType.

```
-- Unit list interval literals
spark-sql> select interval 1 year 2 month;
1-2
-- Quoted interval literals
spark-sql> select interval '1 year 2 month';
1 years 2 months
```

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

Yes but the following sentence in `sql-migration-guide.md` seems to cover this change.
```
  - In Spark 3.2, the unit list interval literals can not mix year-month fields (YEAR and MONTH) and day-time fields (WEEK, DAY, ..., MICROSECOND).
For example, `INTERVAL 1 day 1 hour` is invalid in Spark 3.2. In Spark 3.1 and earlier,
there is no such limitation and the literal returns value of `CalendarIntervalType`.
To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.interval.enabled` to `true`.
```

### How was this patch tested?

Modified existing tests and add new tests.

Closes #33380 from sarutak/fix-interval-constructor.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 71ea25d4f5)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-17 12:23:50 +03:00
Liang-Chi Hsieh 3d423b94a1 [SPARK-35785][SS][FOLLOWUP] Remove ignored test from RocksDBSuite
### What changes were proposed in this pull request?

This patch removes an ignored test from `RocksDBSuite`.

### Why are the changes needed?

The removed test is now ignored. The test itself doesn't look making sense. For example, the condition for capturing exception is never matched. The test runs updates to RocksDB instances at same remote dir with same versions. This doesn't look like a case it will run through in practice.

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

No

### How was this patch tested?

Existing tests.

Closes #33401 from viirya/remove-ignore-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 8009f0dd92)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-17 02:05:06 -07:00
Chao Sun a7c576ee14 [SPARK-36128][SQL] Apply spark.sql.hive.metastorePartitionPruning for non-Hive tables that uses Hive metastore for partition management
### What changes were proposed in this pull request?

In `CatalogFileIndex.filterPartitions`, check the config `spark.sql.hive.metastorePartitionPruning` and don't pushdown predicates to remote HMS if it is false. Instead, fallback to the `listPartitions` API and do the filtering on the client side.

### Why are the changes needed?

Currently the config `spark.sql.hive.metastorePartitionPruning` is only effective for Hive tables, and for non-Hive tables we'd always use the `listPartitionsByFilter` API from HMS client. On the other hand, by default all data source tables also manage their partitions through HMS, when the config `spark.sql.hive.manageFilesourcePartitions` is turned on. Therefore, it seems reasonable to extend the above config for non-Hive tables as well.

In certain cases the remote HMS service could throw exceptions when using the `listPartitionsByFilter` API, which, on the Spark side, is unrecoverable at the current state. Therefore it would be better to allow users to disable the API by using the above config.

For instance, HMS only allow pushdown date column when direct SQL is used instead of JDO for interacting with the underlying RDBMS, and will throw exception otherwise. Even though the Spark Hive client will attempt to recover itself when the exception happens, it only does so when the config `hive.metastore.try.direct.sql` from remote HMS is `false`. There could be cases where the value of `hive.metastore.try.direct.sql` is true but remote HMS still throws exception.

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

Yes now the config `spark.sql.hive.metastorePartitionPruning` is extended for non-Hive tables which use HMS to manage their partition metadata.

### How was this patch tested?

Added a new unit test:
```
build/sbt "hive/testOnly *PruneFileSourcePartitionsSuite -- -z SPARK-36128"
```

Closes #33348 from sunchao/SPARK-36128-by-filter.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 37dc3f9ea7)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-16 13:32:45 -07:00
Jungtaek Lim 4bfcdf38cf [SPARK-34893][SS] Support session window natively
Introduction: this PR is the last 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 proposes to support native session window. Please refer the comments/design doc in SPARK-10816 for more details on the rationalization and design (could be outdated a bit compared to the PR).

The definition of the boundary of "session window" is [the timestamp of start event ~ the timestamp of last event + gap duration). That said, unlike time window, session window is a dynamic window which can expand if new input row is added to the session. To handle expansion of session window, Spark defines session window per input row, and "merge" windows if they can be merged (boundaries are overlapped).

This PR leverages two different approaches on merging session windows:

1. merging session windows with Spark's aggregation logic (a variant of sort aggregation)
2. updating session window for all rows bound to the same session, and applying aggregation logic afterwards

First one is preferable as it outperforms compared to the second one, though it can be only used if merging session window can be applied altogether with aggregation. It is not applicable on all the cases, so second one is used to cover the remaining cases.

This PR also applies the optimization on merging input rows and existing sessions with retaining the order (group keys + start timestamp of session window), leveraging the fact the number of existing sessions per group key won't be huge.

The state format is versioned, so that we can bring a new state format if we find a better one.

### Why are the changes needed?

For now, to deal with sessionization, Spark requires end users to play with (flat)MapGroupsWithState directly which has a couple of major drawbacks:

1. (flat)MapGroupsWithState is lower level API and end users have to code everything in details for defining session window and merging windows
2. built-in aggregate functions cannot be used and end users have to deal with aggregation by themselves
3. (flat)MapGroupsWithState is only available in Scala/Java.

With native support of session window, end users simply use "session_window" like they use "window" for tumbling/sliding window, and leverage built-in aggregate functions as well as UDAFs to simply define aggregations.

Quoting the query example from test suite:

```
    val inputData = MemoryStream[(String, Long)]

    // Split the lines into words, treat words as sessionId of events
    val events = inputData.toDF()
      .select($"_1".as("value"), $"_2".as("timestamp"))
      .withColumn("eventTime", $"timestamp".cast("timestamp"))
      .selectExpr("explode(split(value, ' ')) AS sessionId", "eventTime")
      .withWatermark("eventTime", "30 seconds")

    val sessionUpdates = events
      .groupBy(session_window($"eventTime", "10 seconds") as 'session, 'sessionId)
      .agg(count("*").as("numEvents"))
      .selectExpr("sessionId", "CAST(session.start AS LONG)", "CAST(session.end AS LONG)",
        "CAST(session.end AS LONG) - CAST(session.start AS LONG) AS durationMs",
        "numEvents")
```

which is same as StructuredSessionization (native session window is shorter and clearer even ignoring model classes).

39542bb81f/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala (L66-L105)

(Worth noting that the code in StructuredSessionization only works with processing time. The code doesn't consider old event can update the start time of old session.)

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

Yes. This PR brings the new feature to support session window on both batch and streaming query, which adds a new function "session_window" which usage is similar with "window".

### How was this patch tested?

New test suites. Also tested with benchmark code.

Closes #33081 from HeartSaVioR/SPARK-34893-SPARK-10816-PR-31570-part-5.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit f2bf8b051b)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-16 20:38:35 +09:00
Ke Jia de3b8b996f [SPARK-35710][SQL] Support DPP + AQE when there is no reused broadcast exchange
### What changes were proposed in this pull request?
This PR add the DPP + AQE support when spark can't reuse the broadcast but executing the DPP subquery is cheaper.

### Why are the changes needed?
Improve AQE + DPP

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

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

Closes #32861 from JkSelf/supportDPP3.

Lead-authored-by: Ke Jia <ke.a.jia@intel.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit c1b3f86c58)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-16 16:01:23 +08:00
Steven Aerts 109247f02e [SPARK-35985][SQL] push partitionFilters for empty readDataSchema
this commit makes sure that for File Source V2 partition filters are
also taken into account when the readDataSchema is empty.
This is the case for queries like:

    SELECT count(*) FROM tbl WHERE partition=foo
    SELECT input_file_name() FROM tbl WHERE partition=foo

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

As described in SPARK-35985 there is bug in the File Datasource V2 which prevents it to push down to the FileScanner for queries like the ones listed above.

### Why are the changes needed?

If partitions filters are not pushed down, the whole dataset will be scanned while only one partition is interesting.

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

### How was this patch tested?

An extra test was added which relies on the output of explain, as is done in other places.

Closes #33191 from steven-aerts/SPARK-35985.

Authored-by: Steven Aerts <steven.aerts@airties.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit f06aa4a3f3)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-16 04:53:23 +00:00
Max Gekk 57849a54da [SPARK-36034][SQL] Rebase datetime in pushed down filters to parquet
### What changes were proposed in this pull request?
In the PR, I propose to propagate either the SQL config `spark.sql.parquet.datetimeRebaseModeInRead` or/and Parquet option `datetimeRebaseMode` to `ParquetFilters`. The `ParquetFilters` class uses the settings in conversions of dates/timestamps instances from datasource filters to values pushed via `FilterApi` to the `parquet-column` lib.

Before the changes, date/timestamp values expressed as days/microseconds/milliseconds are interpreted as offsets in Proleptic Gregorian calendar, and pushed to the parquet library as is. That works fine if timestamp/dates values in parquet files were saved in the `CORRECTED` mode but in the `LEGACY` mode, filter's values could not match to actual values.

After the changes, timestamp/dates values of filters pushed down to parquet libs such as `FilterApi.eq(col1, -719162)` are rebased according the rebase settings. For the example, if the rebase mode is `CORRECTED`, **-719162** is pushed down as is but if the current rebase mode is `LEGACY`, the number of days is rebased to **-719164**. For more context, the PR description https://github.com/apache/spark/pull/28067 shows the diffs between two calendars.

### Why are the changes needed?
The changes fix the bug portrayed by the following example from SPARK-36034:
```scala
In [27]: spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "LEGACY")
>>> spark.sql("SELECT DATE '0001-01-01' AS date").write.mode("overwrite").parquet("date_written_by_spark3_legacy")
>>> spark.read.parquet("date_written_by_spark3_legacy").where("date = '0001-01-01'").show()
+----+
|date|
+----+
+----+
```
The result must have the date value `0001-01-01`.

### Does this PR introduce _any_ user-facing change?
In some sense, yes. Query results can be different in some cases. For the example above:
```scala
scala> spark.conf.set("spark.sql.parquet.datetimeRebaseModeInWrite", "LEGACY")
scala> spark.sql("SELECT DATE '0001-01-01' AS date").write.mode("overwrite").parquet("date_written_by_spark3_legacy")
scala> spark.read.parquet("date_written_by_spark3_legacy").where("date = '0001-01-01'").show(false)
+----------+
|date      |
+----------+
|0001-01-01|
+----------+
```

### How was this patch tested?
By running the modified test suite `ParquetFilterSuite`:
```
$ build/sbt "test:testOnly *ParquetV1FilterSuite"
$ build/sbt "test:testOnly *ParquetV2FilterSuite"
```

Closes #33347 from MaxGekk/fix-parquet-ts-filter-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit b09b7f7cc0)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-15 22:22:09 +03:00
Gengliang Wang fce19ab31a [SPARK-36135][SQL] Support TimestampNTZ type in file partitioning
### What changes were proposed in this pull request?

Support TimestampNTZ type in file partitioning
* When there is no provided schema and the default Timestamp type is TimestampNTZ , Spark should infer and parse the timestamp value partitions as TimestampNTZ.
* When the provided Partition schema is TimestampNTZ, Spark should be able to parse the TimestampNTZ type partition column.

### Why are the changes needed?

File partitioning is an important feature and Spark should support TimestampNTZ type in it.

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

Yes, Spark supports TimestampNTZ type in file partitioning

### How was this patch tested?

Unit tests

Closes #33344 from gengliangwang/partition.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 96c2919988)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-16 01:13:49 +08:00
Yuming Wang 1ed72e2e8e [SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT IN predicate
### What changes were proposed in this pull request?

This pr fix Parquet filter pushdown `NOT` `IN` predicate if its values exceeds `spark.sql.parquet.pushdown.inFilterThreshold`. For example: `Not(In(a, Array(2, 3, 7))`. We can not push down `not(and(gteq(a, 2), lteq(a, 7)))`.

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test.

Closes #33365 from wangyum/SPARK-32792-3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 0062c03c15)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-15 18:52:06 +03:00
PengLei 1a3c56d3ea [SPARK-29519][SQL][FOLLOWUP] Keep output is deterministic for show tblproperties
### What changes were proposed in this pull request?
Keep the output order is deterministic for `SHOW TBLPROPERTIES`

### Why are the changes needed?
[#33343](https://github.com/apache/spark/pull/33343#issue-689828187).
Keep the output order deterministic meaningful.

Since the properties are sorted and then compare result in the testcase for `SHOW TBLPROPERTIES`,  it does not fail, but ideally, the output is ordered and deterministic.

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

### How was this patch tested?
existed ut test

Closes #33353 from Peng-Lei/order-ouput-properties.

Authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e05441c223)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-15 21:44:27 +08:00
Kousuke Saruta 42710991e2 [SPARK-33898][SQL][FOLLOWUP] Fix the behavior of SHOW CREATE TABLE to output deterministic results
### What changes were proposed in this pull request?

This PR fixes a behavior of `SHOW CREATE TABLE` added in `SPARK-33898` (#32931) to output deterministic result.
A test `SPARK-33898: SHOW CREATE TABLE` in `DataSourceV2SQLSuite` compares two `CREATE TABLE` statements. One is generated by `SHOW CREATE TABLE` against a created table and the other is expected `CREATE TABLE` statement.

The created table has options `from` and `to`, and they are declared in this order.
```
CREATE TABLE $t (
  a bigint NOT NULL,
  b bigint,
  c bigint,
  `extra col` ARRAY<INT>,
  `<another>` STRUCT<x: INT, y: ARRAY<BOOLEAN>>
)
USING foo
OPTIONS (
  from = 0,
  to = 1)
COMMENT 'This is a comment'
TBLPROPERTIES ('prop1' = '1')
PARTITIONED BY (a)
LOCATION '/tmp'
```

And the expected `CREATE TABLE` in the test code is like as follows.
```
"CREATE TABLE testcat.ns1.ns2.tbl (",
"`a` BIGINT NOT NULL,",
"`b` BIGINT,",
"`c` BIGINT,",
"`extra col` ARRAY<INT>,",
"`<another>` STRUCT<`x`: INT, `y`: ARRAY<BOOLEAN>>)",
"USING foo",
"OPTIONS(",
"'from' = '0',",
"'to' = '1')",
"PARTITIONED BY (a)",
"COMMENT 'This is a comment'",
"LOCATION '/tmp'",
"TBLPROPERTIES(",
"'prop1' = '1')"
```
As you can see, the order of `from` and `to` is expected.
But options are implemented as `Map` so the order of key cannot be kept.

In fact, this test fails with Scala 2.13.
```
[info] - SPARK-33898: SHOW CREATE TABLE *** FAILED *** (515 milliseconds)
[info]   Array("CREATE TABLE testcat.ns1.ns2.tbl (", "`a` BIGINT NOT NULL,", "`b` BIGINT,", "`c` BIGINT,", "`extra col` ARRAY<INT>,", "`<another>` STRUCT<`x`: INT, `y`: ARRAY<BOOLEAN>>)", "USING foo", "OPTIONS(", "'to' = '1',", "'from' = '0')", "PARTITIONED BY (a)", "COMMENT 'This is a comment'", "LOCATION '/tmp'", "TBLPROPERTIES(", "'prop1' = '1')") did not equal Array("CREATE TABLE testcat.ns1.ns2.tbl (", "`a` BIGINT NOT NULL,", "`b` BIGINT,", "`c` BIGINT,", "`extra col` ARRAY<INT>,", "`<another>` STRUCT<`x`: INT, `y`: ARRAY<BOOLEAN>>)", "USING foo", "OPTIONS(", "'from' = '0',", "'to' = '1')", "PARTITIONED BY (a)", "COMMENT 'This is a comment'", "LOCATION '/tmp'", "TBLPROPERTIES(", "'prop1' = '1')") (DataSourceV2SQLSuite.scala:1997)
```
In the current master, the test doesn't fail with Scala 2.12 but it's still non-deterministic.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

I confirmed that the modified test passed with both Scala 2.12 and Scala 2.13 with this change.

Closes #33343 from sarutak/fix-show-create-table-test.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit f95ca31c0f)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-15 20:53:32 +09:00
Gengliang Wang 75ff69a994 [SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on daylight saving time
### What changes were proposed in this pull request?

Only use the zone ids that has no daylight saving for testing `localtimestamp`

### Why are the changes needed?

https://github.com/apache/spark/pull/33346#discussion_r670135296 MaxGekk suggests that we should avoid wrong results if possible.

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

No
### How was this patch tested?

Unit test

Closes #33354 from gengliangwang/FIxDST.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 564d3de7c6)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-15 11:41:04 +03:00
Gengliang Wang 83b37dfaf0 [SPARK-36037][SQL][FOLLOWUP] Fix flaky test for datetime function localtimestamp
### What changes were proposed in this pull request?

The threshold of the test case "datetime function localtimestamp" is small, which leads to flaky test results
https://github.com/gengliangwang/spark/runs/3067396143?check_suite_focus=true

This PR is to increase the threshold for checking two the different current local datetimes from 5ms to 1 second. (The test case of current_timestamp uses 5 seconds)
### Why are the changes needed?

Fix flaky test
### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test

Closes #33346 from gengliangwang/fixFlaky.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 0973397721)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-15 11:32:35 +08:00
Karen Feng 8b35bc4d2b [SPARK-36106][SQL][CORE] Label error classes for subset of QueryCompilationErrors
### What changes were proposed in this pull request?

Adds error classes to some of the exceptions in QueryCompilationErrors.

### Why are the changes needed?

Improves auditing for developers and adds useful fields for users (error class and SQLSTATE).

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

Yes, fills in missing error class and SQLSTATE fields.

### How was this patch tested?

Existing tests and new unit tests.

Closes #33309 from karenfeng/group-compilation-errors-1.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit e92b8ea6f8)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-15 11:43:32 +09:00
ulysses-you 0da71548a5 [SPARK-35639][SQL][FOLLOWUP] Make hasCoalescedPartition return true if something was actually coalesced
### What changes were proposed in this pull request?

Add `CoalescedPartitionSpec(0, 0, _)` check if a `CoalescedPartitionSpec` is coalesced.

### Why are the changes needed?

Fix corner case.

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

yes, UI may be changed

### How was this patch tested?

Add test

Closes #33342 from ulysses-you/SPARK-35639-FOLLOW.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 3819641201)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 22:05:05 +08:00
Chao Sun 4cc7d9b8f1 [SPARK-36123][SQL] Parquet vectorized reader doesn't skip null values correctly
### What changes were proposed in this pull request?

Fix the skipping values logic in Parquet vectorized reader when column index is effective, by considering nulls and only call `ParquetVectorUpdater.skipValues` when the values are non-null.

### Why are the changes needed?

Currently, the Parquet vectorized reader may not work correctly if column index filtering is effective, and the data page contains null values. For instance, let's say we have two columns `c1: BIGINT` and `c2: STRING`, and the following pages:
```
   * c1        500       500       500       500
   *  |---------|---------|---------|---------|
   *  |-------|-----|-----|---|---|---|---|---|
   * c2     400   300   300 200 200 200 200 200
```

and suppose we have a query like the following:
```sql
SELECT * FROM t WHERE c1 = 500
```

this will create a Parquet row range `[500, 1000)` which, when applied to `c2`, will require us to skip all the rows in `[400,500)`. However the current logic for skipping rows is via `updater.skipValues(n, valueReader)` which is incorrect since this skips the next `n` non-null values. In the case when nulls are present, this will not work correctly.

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

No

### How was this patch tested?

Added a new test in `ParquetColumnIndexSuite`.

Closes #33330 from sunchao/SPARK-36123-skip-nulls.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e980c7a840)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 18:14:33 +08:00
Linhong Liu c9813f74e9 [SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range
### What changes were proposed in this pull request?
DATE/TIMESTAMP literals support years 0000 to 9999. However, internally we support a range that is much larger.
We can add or subtract large intervals from a date/timestamp and the system will happily process and display large negative and positive dates.

Since we obviously cannot put this genie back into the bottle the only thing we can do is allow matching DATE/TIMESTAMP literals.

### Why are the changes needed?
make spark more usable and bug fix

### Does this PR introduce _any_ user-facing change?
Yes, after this PR, below SQL will have different results
```sql
select cast('-10000-1-2' as date) as date_col
-- before PR: NULL
-- after PR: -10000-1-2
```

```sql
select cast('2021-4294967297-11' as date) as date_col
-- before PR: 2021-01-11
-- after PR: NULL
```

### How was this patch tested?
newly added test cases

Closes #32959 from linhongliu-db/SPARK-35780-full-range-datetime.

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>
(cherry picked from commit b86645776b)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 18:11:53 +08:00
Jungtaek Lim dcee7a65fd [SPARK-34892][SS] Introduce MergingSortWithSessionWindowStateIterator sorting input rows and rows in state efficiently
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 MergingSortWithSessionWindowStateIterator, which does "merge sort" between input rows and sessions in state based on group key and session's start time.

Note that the iterator does merge sort among input rows and sessions grouped by grouping key. The iterator doesn't provide sessions in state which keys don't exist in input rows. For input rows, the iterator will provide all rows regardless of the existence of matching sessions in state.

MergingSortWithSessionWindowStateIterator 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.

### 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 UT added.

Closes #33077 from HeartSaVioR/SPARK-34892-SPARK-10816-PR-31570-part-4.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit 12a576f175)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-14 18:48:05 +09:00
Fu Chen 5bc06fd7d9 [SPARK-36130][SQL] UnwrapCastInBinaryComparison should skip In expression when in.list contains an expression that is not literal
### What changes were proposed in this pull request?

Fix [comment](https://github.com/apache/spark/pull/32488#issuecomment-879315179)
This PR fix rule `UnwrapCastInBinaryComparison` bug. Rule UnwrapCastInBinaryComparison should skip In expression when in.list contains an expression that is not literal.

- In

Before this pr, the following example will throw an exception.
```scala
  withTable("tbl") {
    sql("CREATE TABLE tbl (d decimal(33, 27)) USING PARQUET")
    sql("SELECT d FROM tbl WHERE d NOT IN (d + 1)")
  }
```
- InSet

As the analyzer guarantee that all the elements in the `inSet.hset` are literal, so this is not an issue for `InSet`.

fbf53dee37/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala (L264-L279)

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

No, only bug fix.

### How was this patch tested?

New test.

Closes #33335 from cfmcgrady/SPARK-36130.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 103d16e868)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 15:57:23 +08:00
Eugene Koifman 78796349d9 [SPARK-35639][SQL] Make hasCoalescedPartition return true if something was actually coalesced
### What changes were proposed in this pull request?
Fix `CustomShuffleReaderExec.hasCoalescedPartition` so that it returns true only if some original partitions got combined

### Why are the changes needed?
W/o this change `CustomShuffleReaderExec` description can report `coalesced` even though partitions are unchanged

### Does this PR introduce _any_ user-facing change?
Yes, the `Arguments` in the node description is now accurate:
```
(16) CustomShuffleReader
Input [3]: [registration#4, sum#85, count#86L]
Arguments: coalesced
```

### How was this patch tested?
Existing tests

Closes #32872 from ekoifman/PRISM-77023-fix-hasCoalescedPartition.

Authored-by: Eugene Koifman <eugene.koifman@workday.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 4033b2a3f4)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 15:48:24 +08:00
gengjiaan 1686cff9a1 [SPARK-36037][SQL] Support ANSI SQL LOCALTIMESTAMP datetime value function
### What changes were proposed in this pull request?
`LOCALTIMESTAMP()` is a datetime value function from ANSI SQL.
The syntax show below:
```
<datetime value function> ::=
    <current date value function>
  | <current time value function>
  | <current timestamp value function>
  | <current local time value function>
  | <current local timestamp value function>
<current date value function> ::=
CURRENT_DATE
<current time value function> ::=
CURRENT_TIME [ <left paren> <time precision> <right paren> ]
<current local time value function> ::=
LOCALTIME [ <left paren> <time precision> <right paren> ]
<current timestamp value function> ::=
CURRENT_TIMESTAMP [ <left paren> <timestamp precision> <right paren> ]
<current local timestamp value function> ::=
LOCALTIMESTAMP [ <left paren> <timestamp precision> <right paren> ]
```

`LOCALTIMESTAMP()` returns the current timestamp at the start of query evaluation as TIMESTAMP WITH OUT TIME ZONE. This is similar to `CURRENT_TIMESTAMP()`.
Note we need to update the optimization rule `ComputeCurrentTime` so that Spark returns the same result in a single query if the function is called multiple times.

### Why are the changes needed?
`CURRENT_TIMESTAMP()` returns the current timestamp at the start of query evaluation.
`LOCALTIMESTAMP()` returns the current timestamp without time zone at the start of query evaluation.
The `LOCALTIMESTAMP` function is an ANSI SQL.
The `LOCALTIMESTAMP` function is very useful.

### Does this PR introduce _any_ user-facing change?
'Yes'. Support new function `LOCALTIMESTAMP()`.

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

Closes #33258 from beliefer/SPARK-36037.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit b4f7758944)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-14 15:39:02 +08:00
Chao Sun b4355608e0 [SPARK-36131][SQL][TEST] Refactor ParquetColumnIndexSuite
### What changes were proposed in this pull request?

Refactor `ParquetColumnIndexSuite` and allow better code reuse.

### Why are the changes needed?

A few methods in the test suite can share the same utility method `checkUnalignedPages` so it's better to do that and remove code duplication.

Additionally, `parquet.enable.dictionary` is tested for both `true` and `false` combination.

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

No

### How was this patch tested?

Existing tests.

Closes #33334 from sunchao/SPARK-35743-test-refactoring.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 7a7b086534)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-13 22:50:07 -07:00
Jungtaek Lim fa8c37acb1 [SPARK-34891][SS] Introduce state store manager for session window in streaming query
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 state store manager for session window in streaming query. Session window in batch query wouldn't need to leverage state store manager.

This PR ensures versioning on state format for state store manager, so that we can apply further optimization after releasing Spark version. StreamingSessionWindowStateManager is a trait defining the available methods in session window state store manager. Its subclasses are classes implementing the trait with versioning.

The format of version 1 leverages the new feature of "prefix match scan" to represent the session windows:

* full key : [ group keys, start time in session window ]
* prefix key [ group keys ]

### 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 #31989 from HeartSaVioR/SPARK-34891-SPARK-10816-PR-31570-part-3.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 0fe2d809d6)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-13 08:58:42 -07:00
Gengliang Wang 3ace01b25b [SPARK-36120][SQL] Support TimestampNTZ type in cache table
### What changes were proposed in this pull request?

Support TimestampNTZ type column in SQL command Cache table

### Why are the changes needed?

Cache table should support the new timestamp type.

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

Yes, the TimemstampNTZ type column can used in `CACHE TABLE`

### How was this patch tested?

Unit test

Closes #33322 from gengliangwang/cacheTable.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 067432705f)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-13 17:24:03 +03:00
Wenchen Fan a0f61ccfe4 [SPARK-36033][SQL][TEST] Validate partitioning requirements in TPCDS tests
### What changes were proposed in this pull request?

Make sure all physical plans of TPCDS queries are valid (satisfy the partitioning requirement).

### Why are the changes needed?

improve test coverage

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

no

### How was this patch tested?

N/A

Closes #33248 from cloud-fan/aqe2.

Lead-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 583173b7cc)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-13 21:17:25 +08:00
Wenchen Fan 017b7d3f0b [SPARK-36074][SQL] Add error class for StructType.findNestedField
### What changes were proposed in this pull request?

This PR adds an INVALID_FIELD_NAME error class for the errors in `StructType.findNestedField`. It also cleans up the code there and adds UT for this method.

### Why are the changes needed?

follow the new error message framework

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

no

### How was this patch tested?

existing tests

Closes #33282 from cloud-fan/error.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-13 21:15:00 +08:00
Max Gekk b4692949f8 [SPARK-35735][SQL][FOLLOWUP] Remove unused method IntervalUtils.checkIntervalStringDataType()
### What changes were proposed in this pull request?
Remove the private method `checkIntervalStringDataType()` from `IntervalUtils` since it hasn't been used anymore after https://github.com/apache/spark/pull/33242.

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

### Does this PR introduce _any_ user-facing change?
No. The method is private, and it existing in code base for short time.

### How was this patch tested?
By existing GAs/tests.

Closes #33321 from MaxGekk/SPARK-35735-remove-unused-method.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 1ba3982d16)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-13 15:11:32 +03:00
Kousuke Saruta 1f8e72f9b1 [SPARK-35749][SPARK-35773][SQL] Parse unit list interval literals as tightest year-month/day-time interval types
### What changes were proposed in this pull request?

This PR allow the parser to parse unit list interval literals like `'3' day '10' hours '3' seconds` or `'8' years '3' months` as `YearMonthIntervalType` or `DayTimeIntervalType`.

### Why are the changes needed?

For ANSI compliance.

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

Yes. I noted the following things in the `sql-migration-guide.md`.

* Unit list interval literals are parsed as `YearMonthIntervaType` or `DayTimeIntervalType` instead of `CalendarIntervalType`.
* `WEEK`, `MILLISECONS`, `MICROSECOND` and `NANOSECOND` are not valid units for unit list interval literals.
* Units of year-month and day-time cannot be mixed like `1 YEAR 2 MINUTES`.

### How was this patch tested?

New tests and modified tests.

Closes #32949 from sarutak/day-time-multi-units.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 8e92ef825a)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-13 18:55:22 +08:00
Gengliang Wang 2781bf0721 [SPARK-36119][SQL] Add new SQL function to_timestamp_ltz
### What changes were proposed in this pull request?

Add new SQL function `to_timestamp_ltz`
syntax:
```
to_timestamp_ltz(timestamp_str_column[, fmt])
to_timestamp_ltz(timestamp_column)
to_timestamp_ltz(date_column)
```

### Why are the changes needed?

As the result of to_timestamp become consistent with the SQL configuration spark.sql.timestmapType and there is already a SQL function to_timestmap_ntz, we need new function to_timestamp_ltz to construct timestamp with local time zone values.

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

Yes, a new function for constructing timestamp with local time zone values

### How was this patch tested?

Unit test

Closes #33318 from gengliangwang/to_timestamp_ltz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 01ddaf3918)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-13 17:38:05 +08:00
allisonwang-db 78c4e3710d [SPARK-35551][SQL] Handle the COUNT bug for lateral subqueries
### What changes were proposed in this pull request?
This PR modifies `DecorrelateInnerQuery` to handle the COUNT bug for lateral subqueries. Similar to SPARK-15370, rewriting lateral subqueries as joins can change the semantics of the subquery and lead to incorrect answers.

However we can't reuse the existing code to handle the count bug for correlated scalar subqueries because it assumes the subquery to have a specific shape (either with Filter + Aggregate or Aggregate as the root node). Instead, this PR proposes a more generic way to handle the COUNT bug. If an Aggregate is subject to the COUNT bug, we insert a left outer domain join between the outer query and the aggregate with a `alwaysTrue` marker and rewrite the final result conditioning on the marker. For example:

```sql
-- t1: [(0, 1), (1, 2)]
-- t2: [(0, 2), (0, 3)]
select * from t1 left outer join lateral (select count(*) from t2 where t2.c1 = t1.c1)
```

Without count bug handling, the query plan is
```
Project [c1#44, c2#45, count(1)#53L]
+- Join LeftOuter, (c1#48 = c1#44)
   :- LocalRelation [c1#44, c2#45]
   +- Aggregate [c1#48], [count(1) AS count(1)#53L, c1#48]
      +- LocalRelation [c1#48]
```
and the answer is wrong:
```
+---+---+--------+
|c1 |c2 |count(1)|
+---+---+--------+
|0  |1  |2       |
|1  |2  |null    |
+---+---+--------+
```

With the count bug handling:
```
Project [c1#1, c2#2, count(1)#10L]
+- Join LeftOuter, (c1#34 <=> c1#1)
   :- LocalRelation [c1#1, c2#2]
   +- Project [if (isnull(alwaysTrue#32)) 0 else count(1)#33L AS count(1)#10L, c1#34]
      +- Join LeftOuter, (c1#5 = c1#34)
         :- Aggregate [c1#1], [c1#1 AS c1#34]
         :  +- LocalRelation [c1#1]
         +- Aggregate [c1#5], [count(1) AS count(1)#33L, c1#5, true AS alwaysTrue#32]
            +- LocalRelation [c1#5]
```
and we have the correct answer:
```
+---+---+--------+
|c1 |c2 |count(1)|
+---+---+--------+
|0  |1  |2       |
|1  |2  |0       |
+---+---+--------+
```

### Why are the changes needed?
Fix a correctness bug with lateral join rewrite.

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

### How was this patch tested?
Added SQL query tests. The results are consistent with Postgres' results.

Closes #33070 from allisonwang-db/spark-35551-lateral-count-bug.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 4f760f2b1f)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-13 17:35:20 +08:00
Liang-Chi Hsieh 19478bdf52 [SPARK-36109][SS][TEST] Check data after adding data to topic in KafkaSourceStressSuite
### What changes were proposed in this pull request?

This patch proposes to check data after adding data to topic in `KafkaSourceStressSuite`.

### Why are the changes needed?

The test logic in `KafkaSourceStressSuite` is not stable. For example, https://github.com/apache/spark/runs/3049244904.

Once we add data to a topic and then delete the topic before checking data, the expected answer is different to retrieved data from the sink.

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

No

### How was this patch tested?

Existing tests.

Closes #33311 from viirya/stream-assert.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 201566cdd5)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-13 01:22:06 -07:00
Wenchen Fan c1d8ccfb64 Revert "[SPARK-35253][SPARK-35398][SQL][BUILD] Bump up the janino version to v3.1.4"
### What changes were proposed in this pull request?

This PR reverts https://github.com/apache/spark/pull/32455 and its followup https://github.com/apache/spark/pull/32536 , because the new janino version has a bug that is not fixed yet: https://github.com/janino-compiler/janino/pull/148

### Why are the changes needed?

avoid regressions

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

no

### How was this patch tested?

existing tests

Closes #33302 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit ae6199af44)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-13 12:14:21 +09:00
Gengliang Wang fba3e90863 [SPARK-36046][SQL] Support new functions make_timestamp_ntz and make_timestamp_ltz
### What changes were proposed in this pull request?

Support new functions make_timestamp_ntz and make_timestamp_ltz
Syntax:
* `make_timestamp_ntz(year, month, day, hour, min, sec)`: Create local date-time from year, month, day, hour, min, sec fields
* `make_timestamp_ltz(year, month, day, hour, min, sec[, timezone])`: Create current timestamp with local time zone from year, month, day, hour, min, sec and timezone fields

### Why are the changes needed?

As the result of `make_timestamp` become consistent with the SQL configuration `spark.sql.timestmapType`, we need these two new functions to construct timestamp literals. They align to the functions [`make_timestamp` and `make_timestamptz`](https://www.postgresql.org/docs/9.4/functions-datetime.html) in PostgreSQL

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

Yes, two new datetime functions: make_timestamp_ntz and make_timestamp_ltz.

### How was this patch tested?

End-to-end tests.

Closes #33299 from gengliangwang/make_timestamp_ntz_ltz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 92bf83ed0a)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-12 22:44:38 +03:00
dgd-contributor 12aecb4330 [SPARK-33603][SQL] Grouping exception messages in execution/command
### What changes were proposed in this pull request?
This PR group exception messages in sql/core/src/main/scala/org/apache/spark/sql/execution/command

### 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 #32951 from dgd-contributor/SPARK-33603_grouping_execution/command.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit d03f71657e)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-13 01:29:03 +08:00
Jungtaek Lim 07011eb779 [SPARK-35861][SS] Introduce "prefix match scan" feature on state store
### What changes were proposed in this pull request?

This PR proposes to introduce a new feature "prefix match scan" on state store, which enables users of state store (mostly stateful operators) to group the keys into logical groups, and scan the keys in the same group efficiently.

For example, if the schema of the key of state store is `[ sessionId | session.start ]`, we can scan with prefix key which schema is `[ sessionId ]` (leftmost 1 column) and retrieve all key-value pairs in state store which keys are matched with given prefix key.

This PR will bring the API changes, though the changes are done in the developer API.

* Registering the prefix key

We propose to make an explicit change to the init() method of StateStoreProvider, as below:

```
def init(
      stateStoreId: StateStoreId,
      keySchema: StructType,
      valueSchema: StructType,
      numColsPrefixKey: Int,
      storeConfs: StateStoreConf,
      hadoopConf: Configuration): Unit
```

Please note that we remove an unused parameter “keyIndexOrdinal” as well. The parameter is coupled with getRange() which we will remove as well. See below for rationalization.

Here we provide the number of columns we take to project the prefix key from the full key. If the operator doesn’t leverage prefix match scan, the value can (and should) be 0, because the state store provider may optimize the underlying storage format which may bring extra overhead.

We would like to apply some restrictions on prefix key to simplify the functionality:

* Prefix key is a part of the full key. It can’t be the same as the full key.
  * That said, the full key will be the (prefix key + remaining parts), and both prefix key and remaining parts should have at least one column.
* We always take the columns from the leftmost sequentially, like “seq.take(nums)”.
* We don’t allow reordering of the columns.
* We only guarantee “equality” comparison against prefix keys, and don’t support the prefix “range” scan.
  * We only support scanning on the keys which match with the prefix key.
  * E.g. We don’t support the range scan from user A to user B due to technical complexity. That’s the reason we can’t leverage the existing getRange API.

As we mentioned, we want to make an explicit change to the init() method of StateStoreProvider which would break backward compatibility, assuming that 3rd party state store providers need to update their code in any way to support prefix match scan. Given RocksDB state store provider is being donated to the OSS and plan to be available in Spark 3.2, the majority of the users would migrate to the built-in state store providers, which would remedy the concerns.

* Scanning key-value pairs matched to the prefix key

We propose to add a new method to the ReadStateStore (and StateStore by inheritance), as below:

```
def prefixScan(prefixKey: UnsafeRow): Iterator[UnsafeRowPair]
```

We require callers to pass the `prefixKey` which would have the same schema with the registered prefix key schema. In other words, the schema of the parameter `prefixKey` should match to the projection of the prefix key on the full key based on the number of columns for the prefix key.

The method contract is clear - the method will return the iterator which will give the key-value pairs whose prefix key is matched with the given prefix key. Callers should only rely on the contract and should not expect any other characteristics based on specific details on the state store provider.

In the caller’s point of view, the prefix key is only used for retrieving key-value pairs via prefix match scan. Callers should keep using the full key to do CRUD.

Note that this PR also proposes to make a breaking change, removal of getRange(), which is never be implemented properly and hence never be called properly.

### Why are the changes needed?

* Introducing prefix match scan feature

Currently, the API in state store is only based on key-value data structure. This lacks on advanced data structures like list-like one, which required us to implement the data structure on our own whenever we need it. We had one in stream-stream join, and we were about to have another one in native session window. The custom implementation of data structure based on the state store API tends to be complicated and has to deal with multiple state stores.

We decided to enhance the state store API a bit to remove the requirement for native session window to implement its own. From the operator of native session window, it will just need to do prefix scan on group key to retrieve all sessions belonging to the group key.

Thanks to adding the feature to the part of state store API, this would enable state store providers to optimize the implementation based on the characteristic. (e.g. We will implement this in RocksDB state store provider via leveraging the characteristic that RocksDB sorts the key by natural order of binary format.)

* Removal of getRange API

Before introducing this we sought the way to leverage getRange, but it's quite hard to implement efficiently, with respecting its method contract. Spark always calls the method with (None, None) parameter and all the state store providers (including built-in) implement it as just calling iterator(), which is not respecting the method contract. That said, we can replace all getRange() usages to iterator(), and remove the API to remove any confusions/concerns.

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

Yes for the end users & maintainers of 3rd party state store provider. They will need to upgrade their state store provider implementations to adopt this change.

### How was this patch tested?

Added UT, and also existing UTs to make sure it doesn't break anything.

Closes #33038 from HeartSaVioR/SPARK-35861.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 094300fa60)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-12 09:07:07 -07:00
Chao Sun 5b2f191228 [SPARK-36056][SQL] Combine readBatch and readIntegers in VectorizedRleValuesReader
### What changes were proposed in this pull request?

Combine `readBatch` and `readIntegers` in `VectorizedRleValuesReader` by having them share the same `readBatchInternal` method.

### Why are the changes needed?

`readBatch` and `readIntegers` share similar code path and this Jira aims to combine them into one method for easier maintenance.

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

No

### How was this patch tested?

Existing tests as this is just a refactoring.

Closes #33271 from sunchao/SPARK-35743-read-integers.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 5edbbd1711)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-12 22:30:37 +08:00
Gengliang Wang 4e9e2f32e8 [SPARK-36072][SQL] TO_TIMESTAMP: return different results based on the default timestamp type
### What changes were proposed in this pull request?

The SQL function TO_TIMESTAMP should return different results based on the default timestamp type:
* when "spark.sql.timestampType" is TIMESTAMP_NTZ, return TimestampNTZType literal
* when "spark.sql.timestampType" is TIMESTAMP_LTZ, return TimestampType literal

This PR also refactor the class GetTimestamp and GetTimestampNTZ to reduce duplicated code.

### Why are the changes needed?

As "spark.sql.timestampType" sets the default timestamp type, the to_timestamp function should behave consistently with it.

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

Yes, when the value of "spark.sql.timestampType" is TIMESTAMP_NTZ, the result type of `TO_TIMESTAMP` is of TIMESTAMP_NTZ type.

### How was this patch tested?

Unit test

Closes #33280 from gengliangwang/to_timestamp.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 32720dd3e1)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-12 10:12:41 +03:00
gengjiaan 5816482868 [SPARK-36044][SQL] Suport TimestampNTZ in functions unix_timestamp/to_unix_timestamp
### What changes were proposed in this pull request?
The functions `unix_timestamp`/`to_unix_timestamp` should be able to accept input of `TimestampNTZType`.

### Why are the changes needed?
The functions `unix_timestamp`/`to_unix_timestamp` should be able to accept input of `TimestampNTZType`.

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

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

Closes #33278 from beliefer/SPARK-36044.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 8738682f6a)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-12 09:55:55 +03:00
Gengliang Wang 09e5bbdfbe [SPARK-36083][SQL] make_timestamp: return different result based on the default timestamp type
### What changes were proposed in this pull request?

The SQL function MAKE_TIMESTAMP should return different results based on the default timestamp type:
* when "spark.sql.timestampType" is TIMESTAMP_NTZ, return TimestampNTZType literal
* when "spark.sql.timestampType" is TIMESTAMP_LTZ, return TimestampType literal

### Why are the changes needed?

As "spark.sql.timestampType" sets the default timestamp type, the make_timestamp function should behave consistently with it.

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

Yes, when the value of "spark.sql.timestampType" is TIMESTAMP_NTZ, the result type of `MAKE_TIMESTAMP` is of TIMESTAMP_NTZ type.

### How was this patch tested?

Unit test

Closes #33290 from gengliangwang/mkTS.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 17ddcc9e82)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-11 20:48:04 +03:00
ulysses-you 2e4929b142 [SPARK-36032][SQL] Use inputPlan instead of currentPhysicalPlan to initialize logical link
### What changes were proposed in this pull request?

Change `currentPhysicalPlan.logicalLink.get` to `inputPlan.logicalLink.get` for initial logical link.

### Why are the changes needed?

At `initialPlan` we may remove some Spark Plan with `queryStagePreparationRules`, if removed Spark Plan is top level node, then we will lose the linked logical node.

Since we support AQE side broadcast join config. It's more common that a join is SMJ at normal planner and changed to BHJ after AQE reOptimize. However, `RemoveRedundantSorts` is applied before reOptimize at `initialPlan`, then a local sort might be removed incorrectly if a join is SMJ at first but changed to BHJ during reOptimize.

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

yes, bug fix

### How was this patch tested?

add test

Closes #33244 from ulysses-you/SPARK-36032.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 484b50cadf)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-08 22:40:03 -07:00
Takuya UESHIN 55111cafd1 [SPARK-36062][PYTHON] Try to capture faulthanlder when a Python worker crashes
### What changes were proposed in this pull request?

Try to capture the error message from the `faulthandler` when the Python worker crashes.

### Why are the changes needed?

Currently, we just see an error message saying `"exited unexpectedly (crashed)"` when the UDFs causes the Python worker to crash by like segmentation fault.
We should take advantage of [`faulthandler`](https://docs.python.org/3/library/faulthandler.html) and try to capture the error message from the `faulthandler`.

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

Yes, when a Spark config `spark.python.worker.faulthandler.enabled` is `true`, the stack trace will be seen in the error message when the Python worker crashes.

```py
>>> def f():
...   import ctypes
...   ctypes.string_at(0)
...
>>> sc.parallelize([1]).map(lambda x: f()).count()
```

```
org.apache.spark.SparkException: Python worker exited unexpectedly (crashed): Fatal Python error: Segmentation fault

Current thread 0x000000010965b5c0 (most recent call first):
  File "/.../ctypes/__init__.py", line 525 in string_at
  File "<stdin>", line 3 in f
  File "<stdin>", line 1 in <lambda>
...
```

### How was this patch tested?

Added some tests, and manually.

Closes #33273 from ueshin/issues/SPARK-36062/faulthandler.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 115b8a180f)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-09 11:31:00 +09:00
Angerszhuuuu 2f54d9eed6 [SPARK-36049][SQL] Remove IntervalUnit
### What changes were proposed in this pull request?
Remove IntervalUnit

### Why are the changes needed?
Clean code

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

### How was this patch tested?
Not need

Closes #33265 from AngersZhuuuu/SPARK-36049.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit fef7e1703c)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-08 23:02:35 +03:00
Gengliang Wang ae62c9d772 [SPARK-36054][SQL] Support group by TimestampNTZ type column
### What changes were proposed in this pull request?

Support group by TimestampNTZ type column

### Why are the changes needed?

It's a basic SQL operation.

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

No, the new timestmap type is not released yet.

### How was this patch tested?

Unit test

Closes #33268 from gengliangwang/agg.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 382b66e267)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-08 22:33:43 +03:00
Gengliang Wang 9103c1fe23 [SPARK-36055][SQL] Assign pretty SQL string to TimestampNTZ literals
### What changes were proposed in this pull request?

Currently the TimestampNTZ literals shows only long value instead of timestamp string in its SQL string and toString result.
Before changes (with default timestamp type as TIMESTAMP_NTZ)
```
– !query
select timestamp '2019-01-01\t'
– !query schema
struct<1546300800000000:timestamp_ntz>
```

After changes:
```
– !query
select timestamp '2019-01-01\t'
– !query schema
struct<TIMESTAMP_NTZ '2019-01-01 00:00:00':timestamp_ntz>
```
### Why are the changes needed?

Make the schema of TimestampNTZ literals readable.

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

No

### How was this patch tested?

Unit test

Closes #33269 from gengliangwang/ntzLiteralString.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit ee945e99cc)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-08 21:43:04 +03:00
PengLei 5ec2ddff6a [SPARK-36012][SQL] Add null flag in SHOW CREATE TABLE
### What changes were proposed in this pull request?
When exec the command `SHOW CREATE TABLE`, we should not lost the info null flag if the table column that
is specified `NOT NULL`

### Why are the changes needed?
[SPARK-36012](https://issues.apache.org/jira/browse/SPARK-36012)

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

### How was this patch tested?
Add UT test for V1 and existed UT for V2

Closes #33219 from Peng-Lei/SPARK-36012.

Authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e071721a51)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-09 01:21:57 +08:00
Wenchen Fan b8d3da16b1 [SPARK-35874][SQL] AQE Shuffle should wait for its subqueries to finish before materializing
### What changes were proposed in this pull request?

Currently, AQE uses a very tricky way to trigger and wait for the subqueries:
1. submitting stage calls `QueryStageExec.materialize`
2. `QueryStageExec.materialize` calls `executeQuery`
3. `executeQuery` does some preparation works, which goes to `QueryStageExec.doPrepare`
4. `QueryStageExec.doPrepare` calls `prepare` of shuffle/broadcast, which triggers all the subqueries in this stage
5. `executeQuery` then calls `waitForSubqueries`, which does nothing because `QueryStageExec` itself has no subqueries
6. then we submit the shuffle/broadcast job, without waiting for subqueries
7. for `ShuffleExchangeExec.mapOutputStatisticsFuture`, it calls `child.execute`, which calls `executeQuery` and wait for subqueries in the query tree of `child`
8. The only missing case is: `ShuffleExchangeExec` itself may contain subqueries(repartition expression) and AQE doesn't wait for it.

A simple fix would be overwriting `waitForSubqueries` in `QueryStageExec`, and forward the request to shuffle/broadcast, but this PR proposes a different and probably cleaner way: we follow `execute`/`doExecute` in `SparkPlan`, and add similar APIs in the AQE version of "execute", which gets a future from shuffle/broadcast.

### Why are the changes needed?

bug fix

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

a query fails without the fix and can run now

### How was this patch tested?

new test

Closes #33058 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 2df67a1a1b)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-09 00:21:08 +08:00
Karen Feng f31cf163d9 [SPARK-35958][CORE] Refactor SparkError.scala to SparkThrowable.java
### What changes were proposed in this pull request?

Refactors the base Throwable trait `SparkError.scala` (introduced in SPARK-34920) an interface `SparkThrowable.java`.

### Why are the changes needed?

- Renaming `SparkError` to `SparkThrowable` better reflect sthat this is the base interface for both `Exception` and `Error`
- Migrating to Java maximizes its extensibility

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

Yes; the base trait has been renamed and the accessor methods have changed (eg. `sqlState` -> `getSqlState()`).

### How was this patch tested?

Unit tests.

Closes #33164 from karenfeng/SPARK-35958.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 71c086eb87)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-08 23:55:11 +08:00
Yuanjian Li 097b667db7 [SPARK-35988][SS] The implementation for RocksDBStateStoreProvider
### What changes were proposed in this pull request?
Add the implementation for the RocksDBStateStoreProvider. It's the subclass of StateStoreProvider that leverages all the functionalities implemented in the RocksDB instance.

### Why are the changes needed?
The interface for the end-user to use the RocksDB state store.

### Does this PR introduce _any_ user-facing change?
Yes. New RocksDBStateStore can be used in their applications.

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

Closes #33187 from xuanyuanking/SPARK-35988.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit 0621e78b5f)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-08 21:02:57 +09:00
Gengliang Wang cafb829c42 [SPARK-36043][SQL][TESTS] Add end-to-end tests with default timestamp type as TIMESTAMP_NTZ
### What changes were proposed in this pull request?

Run end-to-end tests with default timestamp type as TIMESTAMP_NTZ to increase test coverage.

### Why are the changes needed?

Inrease test coverage.
Also, there will be more and more expressions have different behaviors when the default timestamp type is TIMESTAMP_NTZ, for example, `to_timestamp`, `from_json`, `from_csv`, and so on. Having this new test suite helps future developments.

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

No

### How was this patch tested?

CI tests.

Closes #33259 from gengliangwang/ntzTest.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 57342dfc1d)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-08 19:39:17 +08:00
Angerszhuuuu 2776e8aa47 [SPARK-36021][SQL][FOLLOWUP] DT/YM func use field byte to keep consistence
### What changes were proposed in this pull request?
With more thought, all DT/YM function use field byte to keep consistence is better

### Why are the changes needed?
Keep code consistence

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

### How was this patch tested?
Not need

Closes #33252 from AngersZhuuuu/SPARK-36021-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 89aa16b4a8)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-08 12:22:18 +03:00
Kousuke Saruta 429d1780b3 [SPARK-36022][SQL] Respect interval fields in extract
### What changes were proposed in this pull request?

This PR fixes an issue about `extract`.
`Extract` should process only existing fields of interval types. For example:

```
spark-sql> SELECT EXTRACT(MONTH FROM INTERVAL '2021-11' YEAR TO MONTH);
11
spark-sql> SELECT EXTRACT(MONTH FROM INTERVAL '2021' YEAR);
0
```
The last command should fail as the month field doesn't present in INTERVAL YEAR.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

New tests.

Closes #33247 from sarutak/fix-extract-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 39002cb995)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-08 09:41:09 +03:00
Cheng Su 12b29cd41a [SPARK-32577][SQL][TEST][FOLLOWUP] Fix the config value of shuffled hash join for all other test queries
### What changes were proposed in this pull request?

This is the followup from https://github.com/apache/spark/pull/33236#issuecomment-875242730, where we are fixing the config value of shuffled hash join, for all other test queries. Found all configs by searching in https://github.com/apache/spark/search?q=spark.sql.join.preferSortMergeJoin .

### Why are the changes needed?

Fix test to have better test coverage.

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

No.

### How was this patch tested?

Existing tests.

Closes #33249 from c21/join-test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 23943e5e40)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-08 13:17:47 +09:00
Angerszhuuuu 74bfbcd643 [SPARK-36021][SQL] Parse interval literals should support more than 2 digits
### What changes were proposed in this pull request?
For case
```
spark-sql> select interval '123456:12' minute to second;
Error in query:
requirement failed: Interval string must match day-time format of '^(?<sign>[+|-])?(?<minute>\d{1,2}):(?<second>(\d{1,2})(\.(\d{1,9}))?)$': 123456:12, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)

== SQL ==
select interval '123456:12' minute to second
----------------^^^
```

we should support hour/minute/second when for more than 2 digits when parse interval literal string

### Why are the changes needed?
Keep consistence

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

### How was this patch tested?
Added UT

Closes #33231 from AngersZhuuuu/SPARK-36021.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit ea3333a200)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 20:31:40 +03:00
gengjiaan 2fc57bba31 [SPARK-36015][SQL] Support TimestampNTZType in the Window spec definition
### What changes were proposed in this pull request?
The method `WindowSpecDefinition.isValidFrameType` doesn't consider `TimestampNTZType`. We should support it as for `TimestampType`.

### Why are the changes needed?
Support `TimestampNTZType` in the Window spec definition.

### Does this PR introduce _any_ user-facing change?
'Yes'. This PR allows users use  `TimestampNTZType` as the sort spec in window spec definition.

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

Closes #33246 from beliefer/SPARK-36015.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 62ff2add94)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 20:27:16 +03:00
gengjiaan 0c7972ba5f [SPARK-36016][SQL] Support TimestampNTZType in expression ApproxCountDistinctForIntervals
### What changes were proposed in this pull request?
The current `ApproxCountDistinctForInterval`s supports `TimestampType`, but not supports timestamp without time zone yet.
This PR will add the function.

### Why are the changes needed?
`ApproxCountDistinctForInterval` need supports `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'. `ApproxCountDistinctForInterval` accepts `TimestampNTZType`.

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

Closes #33243 from beliefer/SPARK-36016.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit be382a6285)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 20:22:59 +03:00
gengjiaan 25ea296c3c [SPARK-36017][SQL] Support TimestampNTZType in expression ApproximatePercentile
### What changes were proposed in this pull request?
The current `ApproximatePercentile` supports `TimestampType`, but not supports timestamp without time zone yet.
This PR will add the function.

### Why are the changes needed?
`ApproximatePercentile` need supports `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'. `ApproximatePercentile` accepts `TimestampNTZType`.

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

Closes #33241 from beliefer/SPARK-36017.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit cc4463e818)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 12:41:23 +03:00
Angerszhuuuu dd038aacd4 [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by hour to minute and unit case-sensitive issue
### What changes were proposed in this pull request?
When cast `10:10` to interval minute to second,  it can be catch by hour to minute regex, here to 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 #33242 from AngersZhuuuu/SPARK-35735-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 3953754f36)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 12:37:34 +03:00
Cheng Su 08cdd1fbcc [SPARK-32577][SQL][TEST] Fix the config value for shuffled hash join in test in-joins.sql
### What changes were proposed in this pull request?

We found the `in-join.sql` does not test shuffled hash join properly in https://issues.apache.org/jira/browse/SPARK-32577, but didn't find a good way to fix it. Given we now have a test config to enforce shuffled hash join in https://github.com/apache/spark/pull/33182, we can fix the test here now as well.

### Why are the changes needed?

Fix test to have better test coverage.

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

No.

### How was this patch tested?

Reran the test to compare the output, and verified the query plan manually to make sure shuffled hash join being used.

Closes #33236 from c21/join-test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit f3c11595ce)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-07 18:16:45 +09:00
Yuming Wang 007e1c9384 [SPARK-35906][SQL][FOLLOWUP] Recursive remove sort if the maximum number of rows less than or equal to 1
### What changes were proposed in this pull request?

Make it recursive remove sort  if the maximum number of rows less than or equal to 1. For example:
```sql
select a from (select a from values(0, 1) t(a, b) order by a) order by a
```

### Why are the changes needed?

Fix Once strategy's idempotence is broken for batch Eliminate Sorts.

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

No.

### How was this patch tested?

Unit test.

Closes #33240 from wangyum/SPARK-35906-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit ddc5cb9051)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-07 14:27:09 +09:00
Linhong Liu f3ec79990f [SPARK-35984][SQL][TEST] Config to force applying shuffled hash join
### What changes were proposed in this pull request?
Add a config `spark.sql.join.forceApplyShuffledHashJoin` to force applying shuffled hash join
during the join selection.

### Why are the changes needed?
In the `SQLQueryTestSuite`, we want to cover 3 kinds of join (BHJ, SHJ, SMJ) in join.sql. But even
if the `spark.sql.join.preferSortMergeJoin` is set to `false`, shuffled hash join is still not guaranteed.
Thus, we need another config to force the selection.

### Does this PR introduce _any_ user-facing change?
No, only for testing

### How was this patch tested?
newly added tests
Verified all queries in join.sql will use `ShuffledHashJoin` when the config set to `true`

Closes #33182 from linhongliu-db/SPARK-35984-hash-join-config.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 7566db6033)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-07 00:58:30 +08:00
Wenchen Fan 6c1c1af6b4 [SPARK-36020][SQL] Check logical link in remove redundant projects
### What changes were proposed in this pull request?

The `RemoveRedundantProjects` feature can conflict with the AQE broadcast threshold ([PR](https://github.com/apache/spark/pull/32391)) sometimes. After removing the project, the physical plan to logical plan link can be changed and we may have a `Project` above `LogicalQueryStage`. This breaks AQE broadcast threshold, because the stats of `Project` does not have the `isRuntime = true` flag, and thus still use the normal broadcast threshold.

This PR updates `RemoveRedundantProjects` to not remove `ProjectExec` that has a different logical plan link than its child.

### Why are the changes needed?

Make AQE broadcast threshold work in more cases.

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

no

### How was this patch tested?

new tests

Closes #33222 from cloud-fan/aqe2.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 6b3ab8262f)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-06 21:17:51 +08:00
Gengliang Wang 8f267226e4 [SPARK-36025][SQL][TESTS] Reduce the run time of DateExpressionsSuite
### What changes were proposed in this pull request?

Some of the test cases in `DateExpressionsSuite` are quite slow:

- `Hour`: 24s
- `Minute`: 26s
- `Day / DayOfMonth`: 8s
- `Year`: 4s

Each test case has a large loop. We should improve them.

### Why are the changes needed?

Save test running time

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

No

### How was this patch tested?

Verified the run times on local:
- `Hour`: 2s
- `Minute`: 3.2
- `Day / DayOfMonth`:0.5s
- `Year`: 2s

Total reduced time: 54.3s

Closes #33229 from gengliangwang/improveTest.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit d5d1222686)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-06 20:17:20 +08:00
Angerszhuuuu b53d285f72 [SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code about parse string to DT/YM
### What changes were proposed in this pull request?
 Refactor code about parse string to DT/YM intervals.

### Why are the changes needed?
Extracting the common code about parse string to DT/YM should improve code maintenance.

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

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

Closes #33217 from AngersZhuuuu/SPARK-35735-35768.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 26d1bb16bc)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-06 13:51:20 +03:00
Kousuke Saruta 5f383f0102 [SPARK-35999][SQL] Make from_csv/to_csv to handle day-time intervals properly
### What changes were proposed in this pull request?

This PR fixes an issue that `from_csv/to_csv` doesn't handle day-time intervals properly.
`from_csv` throws exception if day-time interval types are given.
```
spark-sql> select from_csv("interval '1 2:3:4' day to second", "a interval day to second");
21/07/03 04:39:13 ERROR SparkSQLDriver: Failed in [select from_csv("interval '1 2:3:4' day to second", "a interval day to second")]
java.lang.Exception: Unsupported type: interval day to second
 at org.apache.spark.sql.errors.QueryExecutionErrors$.unsupportedTypeError(QueryExecutionErrors.scala:775)
 at org.apache.spark.sql.catalyst.csv.UnivocityParser.makeConverter(UnivocityParser.scala:224)
 at org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$valueConverters$1(UnivocityParser.scala:134)
```

Also, `to_csv` doesn't handle day-time interval types properly though any exception is thrown.
The result of `to_csv` for day-time interval types is not ANSI interval compliant form.

```
spark-sql> select to_csv(named_struct("a", interval '1 2:3:4' day to second));
93784000000
```
The result above should be `INTERVAL '1 02:03:04' DAY TO SECOND`.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

New tests.

Closes #33226 from sarutak/csv-dtinterval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit def8bc5c96)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-06 17:38:26 +08:00
Kousuke Saruta 634b2e265c [SPARK-35983][SQL] Allow from_json/to_json for map types where value types are day-time intervals
### What changes were proposed in this pull request?

This PR fixes two issues. One is that `to_json` doesn't support `map` types where value types are `day-time` interval types like:
```
spark-sql> select to_json(map('a', interval '1 2:3:4' day to second));
21/07/06 14:53:58 ERROR SparkSQLDriver: Failed in [select to_json(map('a', interval '1 2:3:4' day to second))]
java.lang.RuntimeException: Failed to convert value 93784000000 (class of class java.lang.Long) with the type of DayTimeIntervalType(0,3) to JSON.
```
The other issue is that even if the issue of `to_json` is resolved, `from_json` doesn't support to convert `day-time` interval string to JSON. So the result of following query will be `null`.
```
spark-sql> select from_json(to_json(map('a', interval '1 2:3:4' day to second)), 'a interval day to second');
{"a":null}
```

### Why are the changes needed?

There should be no reason why day-time intervals cannot used as map value types.
`CalendarIntervalTypes` can do it.

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

No.

### How was this patch tested?

New tests.

Closes #33225 from sarutak/json-dtinterval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit c8ff613c3c)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-06 11:07:10 +03:00
Angerszhuuuu c2ef235419 [SPARK-35972][SQL] When replace ExtractValue in NestedColumnAliasing we should use semanticEquals
### What changes were proposed in this pull request?
Ideally, in SQL query, nested columns should result to GetStructField with non-None name. But there are places that can create GetStructField with None name, such as UnresolvedStar.expand, Dataset encoder stuff, etc.
the current `nestedFieldToAlias` cannot catch it up and will cause job failed.

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT,

Closes #33183 from AngersZhuuuu/SPARK-35972.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 87282f04bf)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-06 00:09:45 -07:00
RoryQi 176b055c12 [SPARK-36011][SQL] Disallow altering permanent views based on temporary views or UDFs
### What changes were proposed in this pull request?
PR #15764 disabled creating permanent views based on temporary views or UDFs.  But AlterViewCommand didn't block temporary objects.

### Why are the changes needed?
More robust view canonicalization.

### Does this PR introduce _any_ user-facing change?
Yes, now if you alter a permanent view based on temporary views or UDFs, the operation will fail.

### How was this patch tested?
Add new unit tests.

Closes #33204 from jerqi/alter_view.

Authored-by: RoryQi <1242949407@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit e0c6b2e965)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-06 14:56:25 +08:00
Gengliang Wang e09feda1d2 [SPARK-35978][SQL] Support non-reserved keyword TIMESTAMP_LTZ
### What changes were proposed in this pull request?

Support new keyword `TIMESTAMP_LTZ`, which can be used for:

- timestamp with local time zone data type in DDL
- timestamp with local time zone data type in Cast clause.
- timestamp with local time zone data type literal

### Why are the changes needed?

Users can use `TIMESTAMP_LTZ` in DDL/Cast/Literals for the timestamp with local time zone type directly. The new keyword is independent of the SQL configuration `spark.sql.timestampType`.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33224 from gengliangwang/TIMESTAMP_LTZ.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit b0b9643cd7)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-06 14:33:36 +08:00
Yuanjian Li 22b303a648 [SPARK-35788][SS] Metrics support for RocksDB instance
### What changes were proposed in this pull request?
Add more metrics for the RocksDB instance. We transform the native states from RocksDB.

### Why are the changes needed?
Improve the usability with more metrics for RocksDB instance.

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

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

Closes #32934 from xuanyuanking/SPARK-35788.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit 9544277b0a)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-06 11:12:37 +09:00
Wenchen Fan 0df89d8999 [SPARK-34302][SQL][FOLLOWUP] More code cleanup
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33113, to do some code cleanup:
1. `UnresolvedFieldPosition` doesn't need to include the field name. We can get it through "context" (`AlterTableAlterColumn.column.name`).
2. Run `ResolveAlterTableCommands` in the main resolution batch, so that the column/field resolution is also unified between v1 and v2 commands (same error message).
3. Fail immediately in `ResolveAlterTableCommands` if we can't resolve the field, instead of waiting until `CheckAnalysis`. We don't expect other rules to resolve fields in ALTER  TABLE commands, so failing immediately is simpler and we can remove duplicated code in `CheckAnalysis`.

### Why are the changes needed?

code simplification.

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

no

### How was this patch tested?

existing tests

Closes #33213 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 8b46e26fc6)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-06 03:43:54 +08:00
Gengliang Wang 1ec37dd164 [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ
### What changes were proposed in this pull request?

Support new keyword TIMESTAMP_NTZ, which can be used for:

- timestamp without time zone data type in DDL
- timestamp without time zone data type in Cast clause.
- timestamp without time zone data type literal

### Why are the changes needed?

Users can use `TIMESTAMP_NTZ` in DDL/Cast/Literals for the timestamp without time zone type directly.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33221 from gengliangwang/timstamp_ntz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 5f44acff3d)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-05 22:30:57 +03:00
Gengliang Wang a9947cbd71 [SPARK-35979][SQL] Return different timestamp literals based on the default timestamp type
### What changes were proposed in this pull request?

For the timestamp literal, it should have the following behavior.
1. When `spark.sql.timestampType` is TIMESTAMP_NTZ: if there is no time zone part, return timestamp without time zone literal; otherwise, return timestamp with local time zone literal

2. When `spark.sql.timestampType` is TIMESTAMP_LTZ: return timestamp with local time zone literal

### Why are the changes needed?

When the default timestamp type is TIMESTAMP_NTZ, the result of type literal should return TIMESTAMP_NTZ when there is no time zone part in the string.

From setion 5.3 "literal" of ANSI SQL standard 2011:
```
27) The declared type of a <timestamp literal> that does not specify <time zone interval> is TIMESTAMP(P) WITHOUT TIME ZONE, where P is the number of digits in <seconds fraction>, if specified, and 0 (zero) otherwise. The declared type of a <timestamp literal> that specifies <time zone interval> is TIMESTAMP(P) WITH TIME ZONE, where P is the number of digits in <seconds fraction>, if specified, and 0 (zero) otherwise.
```
Since we don't have "timestamp with time zone", we use timestamp with local time zone instead.
### Does this PR introduce _any_ user-facing change?

No, the new timestmap type and the default timestamp configuration is not released yet.

### How was this patch tested?

Unit test

Closes #33215 from gengliangwang/tsLiteral.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 2fffec7de8)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-06 00:55:13 +08:00
gengjiaan d3e8c9c78b [SPARK-35664][SQL][FOLLOWUP] Fix incorrect comment for TimestampNTZType
### What changes were proposed in this pull request?
This PR fix the incorrect comment for `TimestampNTZType`.

### Why are the changes needed?
Fix the incorrect comment

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

### How was this patch tested?
No need.

Closes #33218 from beliefer/SPARK-35664-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit c605ba2d46)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-05 18:48:26 +08:00
Kousuke Saruta 544b7e16ac [SPARK-35998][SQL] Make from_csv/to_csv to handle year-month intervals properly
### What changes were proposed in this pull request?

This PR fixes an issue that `from_csv/to_csv` doesn't handle year-month intervals properly.
`from_csv` throws exception if year-month interval types are given.
```
spark-sql> select from_csv("interval '1-2' year to month", "a interval year to month");
21/07/03 04:32:24 ERROR SparkSQLDriver: Failed in [select from_csv("interval '1-2' year to month", "a interval year to month")]
java.lang.Exception: Unsupported type: interval year to month
	at org.apache.spark.sql.errors.QueryExecutionErrors$.unsupportedTypeError(QueryExecutionErrors.scala:775)
	at org.apache.spark.sql.catalyst.csv.UnivocityParser.makeConverter(UnivocityParser.scala:224)
	at org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$valueConverters$1(UnivocityParser.scala:134)
```

Also, `to_csv` doesn't handle year-month interval types properly though any exception is thrown.
The result of `to_csv` for year-month interval types is not ANSI interval compliant form.

```
spark-sql> select to_csv(named_struct("a", interval '1-2' year to month));
14
```
The result above should be `INTERVAL '1-2' YEAR TO MONTH`.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

New tests.

Closes #33210 from sarutak/csv-yminterval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit f4237aff7e)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-05 13:11:03 +03:00
ulysses-you ed7c81dfaa [SPARK-35989][SQL] Only remove redundant shuffle if shuffle origin is REPARTITION_BY_COL in AQE
### What changes were proposed in this pull request?

Skip remove shuffle if it's shuffle origin is not `REPARTITION_BY_COL` in AQE.

### Why are the changes needed?

`REPARTITION_BY_COL` doesn't guarantee the output partitioning number so we can remove it safely in AQE.

For `REPARTITION_BY_NUM`, we should retain the shuffle which partition number is specified by user.
For `REBALANCE_PARTITIONS_BY_COL`, it is a special shuffle used to rebalance partitions so we should not remove it.

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

no

### How was this patch tested?

add test

Closes #33188 from ulysses-you/SPARK-35989.

Lead-authored-by: ulysses-you <ulyssesyou18@gmail.com>
Co-authored-by: ulysses <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 7fe4c4a9ad)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-05 17:10:57 +08:00
Cheng Su 39b3a04bfe [SPARK-35794][SQL] Allow custom plugin for AQE cost evaluator
### What changes were proposed in this pull request?

Current AQE has cost evaluator to decide whether to use new plan after replanning. The current used evaluator is `SimpleCostEvaluator` to make decision based on number of shuffle in the query plan. This is not perfect cost evaluator, and different production environments might want to use different custom evaluators. E.g., sometimes we might want to still do skew join even though it might introduce extra shuffle (trade off resource for better latency), sometimes we might want to take sort into consideration for cost as well. Take our own setting as an example, we are using a custom remote shuffle service (Cosco), and the cost model is more complicated. So We want to make the cost evaluator to be pluggable, and developers can implement their own `CostEvaluator` subclass and plug in dynamically based on configuration.

The approach is to introduce a new config to allow define sub-class name of `CostEvaluator` - `spark.sql.adaptive.customCostEvaluatorClass`. And add `CostEvaluator.instantiate` to instantiate the cost evaluator class in `AdaptiveSparkPlanExec.costEvaluator`.

### Why are the changes needed?

Make AQE cost evaluation more flexible.

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

No but an internal config is introduced - `spark.sql.adaptive.customCostEvaluatorClass` to allow custom implementation of `CostEvaluator`.

### How was this patch tested?

Added unit test in `AdaptiveQueryExecSuite.scala`.

Closes #32944 from c21/aqe-cost.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 044dddf288)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-05 09:06:49 +00:00
Kousuke Saruta 26bcf02833 [SPARK-35982][SQL] Allow from_json/to_json for map types where value types are year-month intervals
### What changes were proposed in this pull request?

This PR fixes two issues. One is that `to_json` doesn't support `map` types where value types are `year-month` interval types like:
```
spark-sql> select to_json(map('a', interval '1-2' year to  month));
21/07/02 11:38:15 ERROR SparkSQLDriver: Failed in [select to_json(map('a', interval '1-2' year to  month))]
java.lang.RuntimeException: Failed to convert value 14 (class of class java.lang.Integer) with the type of YearMonthIntervalType(0,1) to JSON.
```
The other issue is that even if the issue of `to_json` is resolved, `from_json` doesn't support to convert `year-month` interval string to JSON. So the result of following query will be `null`.
```
spark-sql> select from_json(to_json(map('a', interval '1-2' year to month)), 'a interval year to month');
{"a":null}
```

### Why are the changes needed?

There should be no reason why year-month intervals cannot used as map value types.
`CalendarIntervalTypes` can do it.

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

No.

### How was this patch tested?

New tests.

Closes #33181 from sarutak/map-json-yminterval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 6474226852)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-05 10:36:08 +03:00
Gengliang Wang ac1c6aa45c [SPARK-35987][SQL] The ANSI flags of Sum and Avg should be kept after being copied
### What changes were proposed in this pull request?

Make the ANSI flag part of expressions `Sum` and `Average`'s parameter list, instead of fetching it from the sessional SQLConf.

### Why are the changes needed?

For Views, it is important to show consistent results even the ANSI configuration is different in the running session. This is why many expressions like 'Add'/'Divide' making the ANSI flag part of its case class parameter list.

We should make it consistent for the expressions `Sum` and `Average`

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

Yes, the `Sum` and `Average` inside a View always behaves the same, independent of the ANSI model SQL configuration in the current session.

### How was this patch tested?

Existing UT

Closes #33186 from gengliangwang/sumAndAvg.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 51103cdcdd)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-05 12:34:39 +08:00
Wenchen Fan ec84982191 [SPARK-35940][SQL] Refactor EquivalentExpressions to make it more efficient
### What changes were proposed in this pull request?

This PR uses 2 ideas to make `EquivalentExpressions` more efficient:
1. do not keep all the equivalent expressions, we only need a count
2. track the "height" of common subexpressions, to quickly do child-parent sort, and filter out non-child expressions in `addCommonExprs`

This PR also fixes several small bugs (exposed by the refactoring), please see PR comments.

### Why are the changes needed?

code cleanup and small perf improvement

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

no

### How was this patch tested?

existing tests

Closes #33142 from cloud-fan/codegen.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit e6ce220690)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-03 08:28:59 -07:00
Liang-Chi Hsieh 0db6f3a844 [SPARK-35785][SS][FOLLOWUP] Ignore concurrent update and cleanup test
### What changes were proposed in this pull request?

This patch ignores the test "ensure that concurrent update and cleanup consistent versions" in #32933. The test is currently flaky and we will address it later.

### Why are the changes needed?

Unblock other developments.

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

No

### How was this patch tested?

Existing tests.

Closes #33195 from viirya/ignore-rocksdb-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit a6e00ee9d7)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 10:58:59 -07:00
Dongjoon Hyun d7990943c3 [SPARK-35992][BUILD] Upgrade ORC to 1.6.9
### What changes were proposed in this pull request?

This PR aims to upgrade Apache ORC to 1.6.9.

### Why are the changes needed?

This is required to bring ORC-804 in order to fix ORC encryption masking bug.

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

No. This is not released yet.

### How was this patch tested?

Pass the newly added test case.

Closes #33189 from dongjoon-hyun/SPARK-35992.

Lead-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit c55b9fd1e0)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 09:50:00 -07:00
Yuanjian Li ae8f91734d [SPARK-35785][SS][3.2] Cleanup support for RocksDB instance
### What changes were proposed in this pull request?
Add the functionality of cleaning up files of old versions for the RocksDB instance and RocksDBFileManager.

### Why are the changes needed?
Part of the implementation of RocksDB state store.

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

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

Closes #33184 from xuanyuanking/branch-3.2.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-07-02 17:12:00 +09:00
Wenchen Fan c1d8178817 [SPARK-35968][SQL] Make sure partitions are not too small in AQE partition coalescing
### What changes were proposed in this pull request?

By default, AQE will set `COALESCE_PARTITIONS_MIN_PARTITION_NUM` to the spark default parallelism, which is usually quite big. This is to keep the parallelism on par with non-AQE, to avoid perf regressions.

However, this usually leads to many small/empty partitions, and hurts performance (although not worse than non-AQE). Users usually blindly set `COALESCE_PARTITIONS_MIN_PARTITION_NUM` to 1, which makes this config quite useless.

This PR adds a new config to set the min partition size, to avoid too small partitions after coalescing. By default, Spark will not respect the target size, and only respect this min partition size, to maximize the parallelism and avoid perf regression in AQE. This PR also adds a bool config to respect the target size when coalescing partitions, and it's recommended to set it to get better overall performance. This PR also deprecates the `COALESCE_PARTITIONS_MIN_PARTITION_NUM` config.

### Why are the changes needed?

AQE is default on now, we should make the perf better in the default case.

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

yes, a new config.

### How was this patch tested?

new tests

Closes #33172 from cloud-fan/aqe2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 0c9c8ff569)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-02 16:07:46 +08:00
Karen Feng 1fda011d71 [SPARK-35955][SQL] Check for overflow in Average in ANSI mode
### What changes were proposed in this pull request?

Fixes decimal overflow issues for decimal average in ANSI mode, so that overflows throw an exception rather than returning null.

### Why are the changes needed?

Query:

```
scala> import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions._

scala> spark.conf.set("spark.sql.ansi.enabled", true)

scala> val df = Seq(
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 1),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2),
     |  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, "intNum").agg(mean("decNum"))
df2: org.apache.spark.sql.DataFrame = [avg(decNum): decimal(38,22)]

scala> df2.show(40,false)
```

Before:
```
+-----------+
|avg(decNum)|
+-----------+
|null       |
+-----------+
```

After:
```
21/07/01 19:48:31 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 24)
java.lang.ArithmeticException: Overflow in sum of decimals.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.overflowInSumOfDecimalError(QueryExecutionErrors.scala:162)
	at org.apache.spark.sql.errors.QueryExecutionErrors.overflowInSumOfDecimalError(QueryExecutionErrors.scala)
	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 org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
	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:499)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:502)
	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

### How was this patch tested?

Unit test

Closes #33177 from karenfeng/SPARK-35955.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-02 12:41:24 +08:00
Rahul Mahadev 47485a3c2d [SPARK-35897][SS] Support user defined initial state with flatMapGroupsWithState in Structured Streaming
### What changes were proposed in this pull request?
This PR aims to add support for specifying a user defined initial state for arbitrary structured streaming stateful processing using [flat]MapGroupsWithState operator.

### Why are the changes needed?
Users can load previous state of their stateful processing as an initial state instead of redoing the entire processing once again.

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

Yes this PR introduces new API
```
  def mapGroupsWithState[S: Encoder, U: Encoder](
      timeoutConf: GroupStateTimeout,
      initialState: KeyValueGroupedDataset[K, S])(
      func: (K, Iterator[V], GroupState[S]) => U): Dataset[U]

  def flatMapGroupsWithState[S: Encoder, U: Encoder](
      outputMode: OutputMode,
      timeoutConf: GroupStateTimeout,
      initialState: KeyValueGroupedDataset[K, S])(
      func: (K, Iterator[V], GroupState[S]) => Iterator[U])

```

### How was this patch tested?

Through unit tests in FlatMapGroupsWithStateSuite

Closes #33093 from rahulsmahadev/flatMapGroupsWithState.

Authored-by: Rahul Mahadev <rahul.mahadev@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-02 11:53:17 +08:00
Anton Okolnychyi fceabe2372 [SPARK-35779][SQL] Dynamic filtering for Data Source V2
### What changes were proposed in this pull request?

This PR implemented the proposal per [design doc](https://docs.google.com/document/d/1RfFn2e9o_1uHJ8jFGsSakp-BZMizX1uRrJSybMe2a6M) for SPARK-35779.

### Why are the changes needed?

Spark supports dynamic partition filtering that enables reusing parts of the query to skip unnecessary partitions in the larger table during joins. This optimization has proven to be beneficial for star-schema queries which are common in the industry. Unfortunately, dynamic pruning is currently limited to partition pruning during joins and is only supported for built-in v1 sources. As more and more Spark users migrate to Data Source V2, it is important to generalize dynamic filtering and expose it to all v2 connectors.

Please, see the design doc for more information on this effort.

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

Yes, this PR adds a new optional mix-in interface for `Scan` in Data Source V2.

### How was this patch tested?

This PR comes with tests.

Closes #32921 from aokolnychyi/dynamic-filtering-wip.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-01 17:00:12 -07:00
Gengliang Wang a643076d4e [SPARK-35975][SQL] New configuration spark.sql.timestampType for the default timestamp type
### What changes were proposed in this pull request?

Add a new configuration `spark.sql.timestampType`, which configures the default timestamp type of Spark SQL, including SQL DDL and Cast clause. Setting the configuration as `TIMESTAMP_NTZ` will use `TIMESTAMP WITHOUT TIME ZONE` as the default type while putting it as `TIMESTAMP_LTZ` will use `TIMESTAMP WITH LOCAL TIME ZONE`.

The default value of the new configuration is TIMESTAMP_LTZ, which is consistent with previous Spark releases.

### Why are the changes needed?

A new configuration for switching the default timestamp type as timestamp without time zone.

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

No, it's a new feature.

### How was this patch tested?

Unit test

Closes #33176 from gengliangwang/newTsTypeConf.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-01 23:25:18 +03:00
SaurabhChawla ca1217667c [SPARK-35756][SQL] unionByName supports struct having same col names but different sequence
### What changes were proposed in this pull request?

unionByName does not supports struct having same col names but different sequence
```
val df1 = Seq((1, Struct1(1, 2))).toDF("a", "b")
val df2 = Seq((1, Struct2(1, 2))).toDF("a", "b")
val unionDF = df1.unionByName(df2)
```
it gives the exception

`org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. struct<c2:int,c1:int> <> struct<c1:int,c2:int> at the second column of the second table; 'Union false, false :- LocalRelation [_1#38, _2#39] +- LocalRelation _1#45, _2#46`

In this case the col names are same so this unionByName should have the support to check within in the Struct if col names are same it should not throw this exception and works.

after fix we are getting the result

```
val unionDF = df1.unionByName(df2)
scala>  unionDF.show
+---+------+
|  a|     b|
+---+------+
|  1|{1, 2}|
|  1|{2, 1}|
+---+------+

```

### Why are the changes needed?
As per unionByName functionality based on name, does the union. In the case of struct this scenario was missing where all the columns  names are same but sequence is different,  so added this functionality.

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

### How was this patch tested?
Added the unit test and also done the testing through spark shell

Closes #32972 from SaurabhChawla100/SPARK-35756.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 17:37:09 +00:00
Gengliang Wang 3acc4b973b [SPARK-35971][SQL] Rename the type name of TimestampNTZType as "timestamp_ntz"
### What changes were proposed in this pull request?

Rename the type name string of TimestampNTZType from "timestamp without time zone" to "timestamp_ntz".

### Why are the changes needed?

This is to make the column header shorter and simpler.
Snowflake and Flink uses similar approach:
https://docs.snowflake.com/en/sql-reference/data-types-datetime.html
https://ci.apache.org/projects/flink/flink-docs-master/docs/dev/table/concepts/timezone/

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit tests

Closes #33173 from gengliangwang/reviseTypeName.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-01 20:50:19 +08:00
Linhong Liu 3c683434fa [SPARK-35686][SQL] Not allow using auto-generated alias when creating view
### What changes were proposed in this pull request?
As described in  #32831, Spark has compatible issues when querying a view created by an
older version. The root cause is that Spark changed the auto-generated alias name. To avoid
this in the future, we could ask the user to specify explicit column names when creating
a view.

### Why are the changes needed?
Avoid compatible issue when querying a view

### Does this PR introduce _any_ user-facing change?
Yes. User will get error when running query below after this change
```
CREATE OR REPLACE VIEW v AS SELECT CAST(t.a AS INT), to_date(t.b, 'yyyyMMdd') FROM t
```

### How was this patch tested?
not yet

Closes #32832 from linhongliu-db/SPARK-35686-no-auto-alias.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 12:47:38 +00:00
Kent Yao 6699f76fe2 [SPARK-35966][SQL] Port HIVE-17952: Fix license headers to avoid dangling javadoc warnings
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
  8. If you want to add or modify an error message, please read the guideline first:
     https://spark.apache.org/error-message-guidelines.html
-->

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

Port HIVE-17952: Fix license headers to avoid dangling javadoc warnings

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

### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
no

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

Closes #33169 from yaooqinn/SPARK-35966.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-01 18:22:04 +08:00
Linhong Liu 0c34b96541 [SPARK-35685][SQL] Prompt recreating the view when there is an incompatible schema issue
### What changes were proposed in this pull request?
If the user creates a view in 2.4 and reads it in 3.1/3.2, there will be an incompatible schema issue.
So this PR adds a view ddl in the error message to prompt the user recreating the view to fix the
incompatible issue.
For example:
```sql
-- create view in 2.4
CREATE TABLE IF NOT EXISTS t USING parquet AS SELECT '1' as a, '20210420' as b"
CREATE OR REPLACE VIEW v AS SELECT CAST(t.a AS INT), to_date(t.b, 'yyyyMMdd') FROM t
-- select view in master
SELECT * FROM v
```
Then we will get below error:
```
cannot resolve '`to_date(spark_catalog.default.t.b, 'yyyyMMdd')`' given input columns: [a, to_date(b, yyyyMMdd)];
```

### Why are the changes needed?
Improve the error message

### Does this PR introduce _any_ user-facing change?
Yes, the error message will change

### How was this patch tested?
newly added test case

Closes #32831 from linhongliu-db/SPARK-35685-view-compatible.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 09:45:14 +00:00
allisonwang-db f281736fbd [SPARK-35618][SQL] Resolve star expressions in subqueries using outer query plans
### What changes were proposed in this pull request?
This PR supports resolving star expressions in subqueries using outer query plans.

### Why are the changes needed?
Currently, Spark can only resolve star expressions using the inner query plan when resolving subqueries. Instead, it should also be able to resolve star expressions using the outer query plans.

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

### How was this patch tested?
Unit tests

Closes #32787 from allisonwang-db/spark-35618-resolve-star-in-subquery.

Lead-authored-by: allisonwang-db <allison.wang@databricks.com>
Co-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 09:22:55 +00:00
Gengliang Wang f2492772ba [SPARK-35963][SQL] Rename TimestampWithoutTZType to TimestampNTZType
### What changes were proposed in this pull request?

Rename TimestampWithoutTZType to TimestampNTZType

### Why are the changes needed?

The time name of `TimestampWithoutTZType` is verbose. Rename it as `TimestampNTZType` so that
1. it is easier to read and type.
2. As we have the function to_timestamp_ntz, this makes the names consistent.
3. We will introduce a new SQL configuration `spark.sql.timestampType` for the default timestamp type. The configuration values can be "TIMESTMAP_NTZ" or "TIMESTMAP_LTZ" for simplicity.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Run `git grep -i WithoutTZ` and there is no result.
And Ci tests.

Closes #33167 from gengliangwang/rename.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 08:49:15 +00:00
ulysses-you ba0a479bda [SPARK-35961][SQL] Only use local shuffle reader when REBALANCE_PARTITIONS_BY_NONE without CustomShuffleReaderExec
### What changes were proposed in this pull request?

Remove dead code in `OptimizeLocalShuffleReader`.

### Why are the changes needed?

After [SPARK-35725](https://issues.apache.org/jira/browse/SPARK-35725), we might expand partition if that partition is skewed. So the partition number check `bytesByPartitionId.length == partitionSpecs.size` would be wrong if some partitions are coalesced and some partitions are splitted into smaller.
Note that, it's unlikely happened in real world since it used RoundRobin.

Otherhand, after [SPARK-34899](https://issues.apache.org/jira/browse/SPARK-34899), we use origin plan if can not coalesce partitions. So the assuming of that shuffle stage has `CustomShuffleReaderExec` with no effect is always false in `REBALANCE_PARTITIONS_BY_NONE` shuffle origin. That said, if no rule was efficient, there would be no `CustomShuffleReaderExec`.

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

No

### How was this patch tested?

Pass CI

Closes #33165 from ulysses-you/SPARK-35961.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 05:43:11 +00:00
gengjiaan 5d74ace648 [SPARK-35065][SQL] Group exception messages in spark/sql (core)
### What changes were proposed in this pull request?
This PR group all exception messages in `sql/core/src/main/scala/org/apache/spark/sql`.

### 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 #32958 from beliefer/SPARK-35065.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 02:38:06 +00:00
Wenchen Fan cd6a463811 [SPARK-35888][SQL][FOLLOWUP] Return partition specs for all the shuffles
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33079, to fix a bug in corner cases: `ShufflePartitionsUtil.coalescePartitions` should either return the shuffle spec for all the shuffles, or none.

If the input RDD has no partition, the `mapOutputStatistics` is None, and we should still return shuffle specs with size 0.

### 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 #33158 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-01 01:43:11 +00:00
Chao Sun a5c886619d [SPARK-34859][SQL] Handle column index when using vectorized Parquet reader
### What changes were proposed in this pull request?

Make the current vectorized Parquet reader to work with column index introduced in Parquet 1.11. In particular, this PR makes the following changes:
1. in `ParquetReadState`, track row ranges returned via `PageReadStore.getRowIndexes` as well as the first row index for each page via `DataPage.getFirstRowIndex`.
1. introduced a new API `ParquetVectorUpdater.skipValues` which skips a batch of values from a Parquet value reader. As part of the process also renamed existing `updateBatch` to `readValues`, and `update` to `readValue` to keep the method names consistent.
1. in correspondence as above, also introduced new API `VectorizedValuesReader.skipXXX` for different data types, as well as the implementations. These are useful when the reader knows that the given batch of values can be skipped, for instance, due to the batch is not covered in the row ranges generated by column index filtering.
2. changed `VectorizedRleValuesReader` to handle column index filtering. This is done by comparing the range that is going to be read next within the current RLE/PACKED block (let's call this block range), against the current row range. There are three cases:
    * if the block range is before the current row range, skip all the values in the block range
    * if the block range is after the current row range, advance the row range and repeat the steps
    * if the block range overlaps with the current row range, only read the values within the overlapping area and skip the rest.

### Why are the changes needed?

[Parquet Column Index](https://github.com/apache/parquet-format/blob/master/PageIndex.md) is a new feature in Parquet 1.11 which allows very efficient filtering on page level (some benchmark numbers can be found [here](https://blog.cloudera.com/speeding-up-select-queries-with-parquet-page-indexes/)), especially when data is sorted. The feature is largely implemented in parquet-mr (via classes such as `ColumnIndex` and `ColumnIndexFilter`). In Spark, the non-vectorized Parquet reader can automatically benefit from the feature after upgrading to Parquet 1.11.x, without any code change. However, the same is not true for vectorized Parquet reader since Spark chose to implement its own logic such as reading Parquet pages, handling definition levels, reading values into columnar batches, etc.

Previously, [SPARK-26345](https://issues.apache.org/jira/browse/SPARK-26345) / (#31393) updated Spark to only scan pages filtered by column index from parquet-mr side. This is done by calling `ParquetFileReader.readNextFilteredRowGroup` and `ParquetFileReader.getFilteredRecordCount` API. The implementation, however, only work for a few limited cases: in the scenario where there are multiple columns and their type width are different (e.g., `int` and `bigint`), it could return incorrect result. For this issue, please see SPARK-34859 for a detailed description.

In order to fix the above, Spark needs to leverage the API `PageReadStore.getRowIndexes` and `DataPage.getFirstRowIndex`. The former returns the indexes of all rows (note the difference between rows and values: for flat schema there is no difference between the two, but for nested schema they're different) after filtering within a Parquet row group. The latter returns the first row index within a single data page. With the combination of the two, one is able to know which rows/values should be filtered while scanning a Parquet page.

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

Yes. Now the vectorized Parquet reader should work correctly with column index.

### How was this patch tested?

Borrowed tests from #31998 and added a few more tests.

Closes #32753 from sunchao/SPARK-34859.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Li Xian <lxian2shell@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-30 14:21:18 -07:00
ulysses-you d46c1e38ec [SPARK-35725][SQL] Support optimize skewed partitions in RebalancePartitions
### What changes were proposed in this pull request?

* Add a new rule `ExpandShufflePartitions` in AQE `queryStageOptimizerRules`
* Add a new config `spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled` to decide if should enable the new rule

The new rule `OptimizeSkewInRebalancePartitions` only handle two shuffle origin `REBALANCE_PARTITIONS_BY_NONE` and `REBALANCE_PARTITIONS_BY_COL` for data skew issue. And re-use the exists config `ADVISORY_PARTITION_SIZE_IN_BYTES` to decide what partition size should be.

### Why are the changes needed?

Currently, we don't support expand partition dynamically in AQE which is not friendly for some data skew job.

Let's say if we have a simple query:
```
SELECT /*+ REBALANCE(col) */ * FROM table
```

The column of `col` is skewed, then some shuffle partitions would handle too much data than others.

If we haven't inroduced extra shuffle, we can optimize this case by expanding partitions in AQE.

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

Yes, a new config

### How was this patch tested?

Add test

Closes #32883 from ulysses-you/expand-partition.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-30 18:04:50 +00:00
Gengliang Wang 733e85f1f4 [SPARK-35953][SQL] Support extracting date fields from timestamp without time zone
### What changes were proposed in this pull request?

Support extracting date fields from timestamp without time zone, which includes:
- year
- month
- day
- year of week
- week
- day of week
- quarter
- day of month
- day of year

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

No, the timestamp without time zone type is not released yet.

### How was this patch tested?

Unit tests

Closes #33156 from gengliangwang/dateField.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-01 00:44:48 +08:00
Angerszhuuuu 2febd5c3f0 [SPARK-35735][SQL] Take into account day-time interval fields in cast
### What changes were proposed in this pull request?
Support take into account day-time interval field in cast.

### Why are the changes needed?
To conform to the SQL standard.

### Does this PR introduce _any_ user-facing change?
An user can use `cast(str, DayTimeInterval(DAY, HOUR))`, for instance.

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

Closes #32943 from AngersZhuuuu/SPARK-35735.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-30 16:05:04 +03:00
Gengliang Wang e88aa49287 [SPARK-35932][SQL] Support extracting hour/minute/second from timestamp without time zone
### What changes were proposed in this pull request?

Support extracting hour/minute/second fields from timestamp without time zone values. In details, the following syntaxes are supported:

- extract [hour | minute | second] from timestampWithoutTZ
- date_part('[hour | minute | second]', timestampWithoutTZ)
- hour(timestampWithoutTZ)
- minute(timestampWithoutTZ)
- second(timestampWithoutTZ)

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

No, the timestamp without time zone type is not release yet.

### How was this patch tested?

Unit test

Closes #33136 from gengliangwang/field.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-30 19:36:47 +08:00
Karen Feng e3bd817d65 [SPARK-34920][CORE][SQL] Add error classes with SQLSTATE
### What changes were proposed in this pull request?

Unifies exceptions thrown from Spark under a single base trait `SparkError`, which unifies:
- Error classes
- Parametrized error messages
- SQLSTATE, as discussed in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Add-error-IDs-td31126.html.

### Why are the changes needed?

- Adding error classes creates a consistent label for exceptions, even as error messages change
- Creating a single, centralized source-of-truth for parametrized error messages improves auditing for error message quality
- Adding SQLSTATE helps ODBC/JDBC users receive standardized error codes

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

Yes, changes ODBC experience by:
- Adding error classes to error messages
- Adding SQLSTATE to TStatus

### How was this patch tested?

Unit tests, as well as local tests with PyODBC.

Closes #32850 from karenfeng/SPARK-34920.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-30 09:22:02 +00:00
Max Gekk d28ca9cc98 [SPARK-35935][SQL] Prevent failure of MSCK REPAIR TABLE on table refreshing
### What changes were proposed in this pull request?
In the PR, I propose to catch all non-fatal exceptions coming `refreshTable()` at the final stage of table repairing, and output an error message instead of failing with an exception.

### Why are the changes needed?
1. The uncaught exceptions from table refreshing might be considered as regression comparing to previous Spark versions. Table refreshing was introduced by https://github.com/apache/spark/pull/31066.
2. This should improve user experience with Spark SQL. For instance, when the `MSCK REPAIR TABLE` is performed in a chain of command in SQL where catching exception is difficult or even impossible.

### Does this PR introduce _any_ user-facing change?
Yes. Before the changes the `MSCK REPAIR TABLE` command can fail with the exception portrayed in SPARK-35935. After the changes, the same command outputs error message, and completes successfully.

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

Closes #33137 from MaxGekk/msck-repair-catch-except.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-30 09:44:52 +03:00
Max Gekk 76682268d7 Revert "[SPARK-33995][SQL] Expose make_interval as a Scala function"
### What changes were proposed in this pull request?
This reverts commit e6753c9402.

### Why are the changes needed?
The `make_interval` function aims to construct values of the legacy interval type `CalendarIntervalType` which will be substituted by ANSI interval types (see SPARK-27790). Since the function has not been released yet, it would be better to don't expose it via public API at all.

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

### How was this patch tested?
By existing test suites, and GA/jenkins builds.

Closes #33143 from MaxGekk/revert-make_interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-30 09:26:35 +03:00
Gengliang Wang ad4b6796f6 [SPARK-35937][SQL] Extracting date field from timestamp should work in ANSI mode
### What changes were proposed in this pull request?

Add a new ANSI type coercion rule: when getting a date field from a Timestamp column, cast the column as Date type.

This is Spark's current hack to make the implementation simple. In the default type coercion rules, the implicit cast rule does the work. However, The ANSI implicit cast rule doesn't allow converting Timestamp type as Date type, so we need to have this additional rule to make sure the date field extraction from Timestamp columns works.

### Why are the changes needed?

Fix a bug.

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

No, the new type coercion rules are not released yet.

### How was this patch tested?

Unit test

Closes #33138 from gengliangwang/fixGetDateField.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-30 13:53:51 +08:00
Liang-Chi Hsieh 064230de97 [SPARK-35829][SQL] Clean up evaluates subexpressions and add more flexibility to evaluate particular subexpressoin
### What changes were proposed in this pull request?

This patch refactors the evaluation of subexpressions.

There are two changes:

1. Clean up subexpression code after evaluation to avoid duplicate evaluation.
2. Evaluate all children subexpressions when evaluating a subexpression.

### Why are the changes needed?

Currently `subexpressionEliminationForWholeStageCodegen` return the gen-ed code of subexpressions. The caller simply puts the code into its code block. We need more flexible evaluation here. For example, for Filter operator's subexpression evaluation, we may need to evaluate particular subexpression for one predicate. Current approach cannot satisfy the requirement.

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

No

### How was this patch tested?

Existing tests.

Closes #32980 from viirya/subexpr-eval.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-29 22:14:37 -07:00
Venki Korukanti 24b67ca9a8 [SPARK-35896][SS] Include more granular metrics for stateful operators in StreamingQueryProgress
### What changes were proposed in this pull request?

Currently the `StateOperatorProgress` in `StreamingQueryProgress` is missing few metrics.

### Why are the changes needed?

The main motivation is find hotspots and have better visibility in the stateful operations. Detailed explanations are in [SPARK-35896](https://issues.apache.org/jira/browse/SPARK-35896).

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

Yes. The `StateOperatorProgress` entries within `StreamingQueryProgress` now contain additional fields as listed in [SPARK-35896](https://issues.apache.org/jira/browse/SPARK-35896). Example `StreamingQueryProgress` output in JSON form.
Before:
```
{

  "id" : "510be3cd-a955-4faf-8456-d97c78d39af5",
  ....
  "durationMs" : {
    "triggerExecution" : 2856,
    ....
  },
  "stateOperators" : [ {
    "numRowsTotal" : 1,
    "numRowsUpdated" : 1,
    "numRowsDroppedByWatermark" : 0,
    "customMetrics" : {
      "loadedMapCacheHitCount" : 0,
      "loadedMapCacheMissCount" : 0,
      "stateOnCurrentVersionSizeBytes" : 392
    }
  }],
  ....
}
```
After:
```
{
  "id" : "510be3cd-a955-4faf-8456-d97c78d39af5",
  ....
  "durationMs" : {
    "triggerExecution" : 2856,
    ....
  },
  "stateOperators" : [ {
    "operatorName" : "dedupe", <-- new
    "numRowsTotal" : 1,
    "numRowsUpdated" : 1, <-- new
    "allUpdatesTimeMs" : 56, <-- new
    "numRowsRemoved" : 2, <-- new
    "allRemovalsTimeMs" : 45, <-- new
    "commitTimeMs" : 40, <-- new
    "numRowsDroppedByWatermark" : 0,
    "numShufflePartitions" : 2, <-- new
    "numStateStoreInstances" : 2, <-- new
    "customMetrics" : {
      "loadedMapCacheHitCount" : 0,
      "loadedMapCacheMissCount" : 0,
      "stateOnCurrentVersionSizeBytes" : 392
    }
  }],
  ....
}
```

### How was this patch tested?

Existing tests for regressions. Added new UTs.

Closes #33091 from vkorukanti/SPARK-35896.

Lead-authored-by: Venki Korukanti <venki.korukanti@gmail.com>
Co-authored-by: Venki Korukanti <venki.korukanti@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-30 13:41:26 +09:00
Yuanjian Li 3257a30e53 [SPARK-35784][SS] Implementation for RocksDB instance
### What changes were proposed in this pull request?
The implementation for the RocksDB instance, which is used in the RocksDB state store. It plays a role as a handler for the RocksDB instance and RocksDBFileManager.

### Why are the changes needed?
Part of the RocksDB state store implementation.

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

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

Closes #32928 from xuanyuanking/SPARK-35784.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-29 17:46:45 -07:00
Yuming Wang 4a17e7a5ae [SPARK-35906][SQL] Remove order by if the maximum number of rows less than or equal to 1
### What changes were proposed in this pull request?

This PR removes order by if the maximum number of rows less than or equal to 1. For example:
```scala
spark.sql("select count(*) from range(1, 10, 2, 2) order by 1 limit 10").explain("cost")
```
Before this pr:
```
== Optimized Logical Plan ==
Sort [count(1)#2L ASC NULLS FIRST], true, Statistics(sizeInBytes=16.0 B)
+- Aggregate [count(1) AS count(1)#2L], Statistics(sizeInBytes=16.0 B, rowCount=1)
   +- Project, Statistics(sizeInBytes=20.0 B)
      +- Range (1, 10, step=2, splits=Some(2)), Statistics(sizeInBytes=40.0 B, rowCount=5)
```

After this pr:
```
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#2L], Statistics(sizeInBytes=16.0 B, rowCount=1)
+- Project, Statistics(sizeInBytes=20.0 B)
   +- Range (1, 10, step=2, splits=Some(2)), Statistics(sizeInBytes=40.0 B, rowCount=5)
```

### 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 #33100 from wangyum/SPARK-35906.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-29 11:04:54 -07:00
Dongjoon Hyun 7e7028282c [SPARK-35928][BUILD] Upgrade ASM to 9.1
### What changes were proposed in this pull request?

This PR aims to upgrade ASM to 9.1

### Why are the changes needed?

The latest `xbean-asm9-shaded` is built with ASM 9.1.

- https://mvnrepository.com/artifact/org.apache.xbean/xbean-asm9-shaded/4.20
- 5e0e3c0c64/pom.xml (L67)

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

No.

### How was this patch tested?

Pass the CIs.

Closes #33130 from dongjoon-hyun/SPARK-35928.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-29 10:27:51 -07:00
ulysses-you def738365e [SPARK-35923][SQL] Coalesce empty partition with mixed CoalescedPartitionSpec and PartialReducerPartitionSpec
### What changes were proposed in this pull request?

Skip empty partitions in `ShufflePartitionsUtil.coalescePartitionsWithSkew`.

### Why are the changes needed?

Since [SPARK-35447](https://issues.apache.org/jira/browse/SPARK-35447), we apply `OptimizeSkewedJoin` before `CoalesceShufflePartitions`. However, There are something different with the order of these two rules.

Let's say if we have a skewed partitions: [0, 128MB, 0, 128MB, 0]:

* coalesce partitions first then optimize skewed partitions:
  [64MB, 64MB, 64MB, 64MB]
* optimize skewed partition first then coalesce partitions:
  [0, 64MB, 64MB, 0, 64MB, 64MB, 0]

So we can do coalesce in `ShufflePartitionsUtil.coalescePartitionsWithSkew` with mixed `CoalescedPartitionSpec` and `PartialReducerPartitionSpec` if `CoalescedPartitionSpec` is empty.

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

No, not release yet.

### How was this patch tested?

Add test.

Closes #33123 from ulysses-you/SPARK-35923.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-29 14:58:51 +00:00
Gengliang Wang 78e6263cce [SPARK-35927][SQL] Remove type collection AllTimestampTypes
### What changes were proposed in this pull request?

Replace the type collection `AllTimestampTypes` with the new data type `AnyTimestampType`

### Why are the changes needed?

As discussed in https://github.com/apache/spark/pull/33115#discussion_r659866760, it is more convenient to have a new data type "AnyTimestampType" instead of using type collection `AllTimestampTypes`:
1. simplify the pattern match
2. In the default type coercion rules, when implicit casting a type to a TypeCollection type, Spark chooses the first convertible data type as the result. If we are going to make the default timestamp type configurable, having AnyTimestampType is better

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

No

### How was this patch tested?

Existing UT

Closes #33129 from gengliangwang/allTimestampTypes.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-29 16:57:08 +08:00
Gengliang Wang 7635114d53 [SPARK-35916][SQL] Support subtraction among Date/Timestamp/TimestampWithoutTZ
### What changes were proposed in this pull request?

Support the following operations:

- TimestampWithoutTZ - Date
- Date - TimestampWithoutTZ
- TimestampWithoutTZ - Timestamp
- Timestamp - TimestampWithoutTZ
- TimestampWithoutTZ - TimestampWithoutTZ

For subtraction between `TimestampWithoutTZ` and `Timestamp`, the `Timestamp` column is cast as TimestampWithoutTZType.

### Why are the changes needed?

Support basic subtraction among Date/Timestamp/TimestampWithoutTZ.

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

No, the timestamp without time zone type is not release yet.

### How was this patch tested?

Unit tests

Closes #33115 from gengliangwang/subtractTimestampWithoutTz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-29 14:45:09 +08:00
Anton Okolnychyi 8a21d2dcfe [SPARK-35899][SQL][FOLLOWUP] Utility to convert connector expressions to Catalyst
### What changes were proposed in this pull request?

This PR addresses post-review comments on PR #33096:
- removes `private[sql]` modifier
- removes the option to pass a resolver to simplify the API

### Why are the changes needed?

These changes are needed to simply the utility API.

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

No.

### How was this patch tested?

Existing tests.

Closes #33120 from aokolnychyi/spark-35899-follow-up.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-28 22:22:07 -07:00
Kousuke Saruta 880bbd6aaa [SPARK-35876][SQL] ArraysZip should retain field names to avoid being re-written by analyzer/optimizer
### What changes were proposed in this pull request?

This PR fixes an issue that field names of structs generated by `arrays_zip` function could be unexpectedly re-written by analyzer/optimizer.
Here is an example.
```
val df = sc.parallelize(Seq((Array(1, 2), Array(3, 4)))).toDF("a1", "b1").selectExpr("arrays_zip(a1, b1) as zipped")
df.printSchema
root
 |-- zipped: array (nullable = true)
 |    |-- element: struct (containsNull = false)
 |    |    |-- a1: integer (nullable = true)                                      // OK. a1 is expected name
 |    |    |-- b1: integer (nullable = true)                                      // OK. b1 is expected name

df.explain
== Physical Plan ==
*(1) Project [arrays_zip(_1#3, _2#4) AS zipped#12]               // Not OK. field names are re-written as _1 and _2 respectively

df.write.parquet("/tmp/test.parquet")
val df2 = spark.read.parquet("/tmp/test.parquet")

df2.printSchema
root
 |-- zipped: array (nullable = true)
 |    |-- element: struct (containsNull = true)
 |    |    |-- _1: integer (nullable = true)                                      // Not OK. a1 is expected but got _1
 |    |    |-- _2: integer (nullable = true)                                      // Not OK. b1 is expected but got _2
```

This issue happens when aliases are eliminated by `AliasHelper.replaceAliasButKeepName` or `AliasHelper.trimNonTopLevelAliases` called via analyzer/optimizer
b89cd8d75a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala (L883)
b89cd8d75a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala (L3759)
I investigated functions which can be affected this issue but I found only `arrays_zip` so far.

To fix this issue, this PR changes the definition of `ArraysZip` to retain field names to avoid being re-written by analyzer/optimizer.

### Why are the changes needed?

This is apparently a bug.

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

No. After this change, the field names are no longer re-written but it should be expected behavior for users.

### How was this patch tested?

New tests.

Closes #33106 from sarutak/arrays-zip-retain-names.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-29 12:28:41 +09:00
Terry Kim 620fde4767 [SPARK-34302][SQL] Migrate ALTER TABLE ... CHANGE COLUMN command to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following `ALTER TABLE ... CHANGE COLUMN` command to use `UnresolvedTable` as a `child` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### Why are the changes needed?

This is a part of effort to make the relation lookup behavior consistent: [SPARK-29900](https://issues.apache.org/jira/browse/SPARK-29900).

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

After this PR, the above `ALTER TABLE ... CHANGE COLUMN` commands will have a consistent resolution behavior.

### How was this patch tested?

Updated existing tests.

Closes #33113 from imback82/alter_change_column.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-29 02:53:05 +00:00
ulysses-you 622fc686e2 [SPARK-35888][SQL] Add dataSize field in CoalescedPartitionSpec
### What changes were proposed in this pull request?

* add `dataSize` field in `CoalescedPartitionSpec`
* add data size test suite in `ShufflePartitionsUtilSuite`

### Why are the changes needed?

Currently, all test suite about `CoalescedPartitionSpec` do not check the data size due to it doesn't contains data size field.

We can add data size in `CoalescedPartitionSpec` and then add test case for better coverage.

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

No

### How was this patch tested?

Pass CI

Closes #33079 from ulysses-you/SPARK-35888.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-29 02:51:24 +00:00
PengLei 8fbbd2e6d7 [SPARK-33898][SQL] Support SHOW CREATE TABLE In V2
### What changes were proposed in this pull request?
1. Implement V2 execution node `ShowCreateTableExec` similar to V1 `ShowCreateTableCommand`
2. No support `SHOW CREATE TABLE XXX AS SERDE`

### Why are the changes needed?
[SPARK-33898](https://issues.apache.org/jira/browse/SPARK-33898)

### Does this PR introduce _any_ user-facing change?
Yes. Support the user to execute `SHOW CREATE TABLE` command in V2 table

### How was this patch tested?
Add two UT tests
1. ./dev/scalastyle
2. run test DataSourceV2SQLSuite

Closes #32931 from Peng-Lei/SPARK-33898.

Lead-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: PengLei <peng.8lei@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-06-29 10:14:46 +08:00
Tom van Bussel c6606502a2 [SPARK-35898][SQL] Fix arrays and maps in RowToColumnConverter
### What changes were proposed in this pull request?

This PR fixes support for arrays and maps in `RowToColumnConverter`. In particular this PR fixes two bugs:

1. `appendArray` in `WritableColumnVector` does not reserve any elements in its child arrays, which causes the assertion in `OffHeapColumnVector.putArray` to fail.
2. The nullability of the child columns is propagated incorrectly when creating the child converters of `ArrayConverter` and `MapConverter` in `RowToColumnConverter`.

This PR fixes these issues.

### Why are the changes needed?

Both bugs cause an exception to be thrown.

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

No

### How was this patch tested?

I added additional test cases to `ColumnVectorSuite` to catch the first bug, and I added `RowToColumnConverterSuite` to catch the both bugs (but specifically the second).

Closes #33108 from tomvanbussel/SPARK-35898.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-06-28 16:50:53 +02:00
PengLei 356aef48b8 [SPARK-35728][SPARK-35778][SQL][TESTS] Check multiply/divide of day-time and year-month interval of any fields by a numeric
### What changes were proposed in this pull request?
[SPARK-35728](https://issues.apache.org/jira/browse/SPARK-35728): Add test case to check multiply/divide of day-time
intervals of any fields by numeric
[SPARK-35778](https://issues.apache.org/jira/browse/SPARK-35778): Add test case to check multiply/divide of year-month intervals of any fields by numeric

### Why are the changes needed?
Improve test coverage

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

### How was this patch tested?
Add ut tests

Lead-authored-by: Lei Peng <peng.8leigmail.com>
Co-authored-by: AngersZhuuuu <angers.zhugmail.com>

Closes #33080 from Peng-Lei/SPARK-35728-35778.

Lead-authored-by: PengLei <peng.8lei@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-28 13:35:54 +03:00
Yuming Wang 108635af17 Revert "[SPARK-35904][SQL] Collapse above RebalancePartitions"
This reverts commit def29e50
2021-06-28 16:23:23 +08:00
dgd-contributor 1c81ad2029 [SPARK-35064][SQL] Group error in spark-catalyst
### What changes were proposed in this pull request?
This PR group exception messages in sql/catalyst/src/main/scala/org/apache/spark/sql (except 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 #32916 from dgd-contributor/SPARK-35064_catalyst_group_error.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-28 07:21:24 +00:00
RoryQi 378ac78bdf [SPARK-35318][SQL][FOLLOWUP] Hide the internal view properties for show tblproperties
### What changes were proposed in this pull request?
PR #32441 hid the internal view properties for describe table command, But the `show tblproperties view` case is not covered.

### 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  `show tblproperties test_view`
```
....
p1 v1
p2 v2
view.catalogAndNamespace.numParts	2
view.catalogAndNamespace.part.0	spark_catalog
view.catalogAndNamespace.part.1	default
view.query.out.col.0	c1
view.query.out.numCols	1
view.referredTempFunctionsNames	[]
view.referredTempViewNames	[]
...
```
After this change, the internal properties will be hidden.
```
....
p1 v1
p2 v2
...
```
### How was this patch tested?
existing UT

Closes #33016 from jerqi/hide_show_tblproperties.

Authored-by: RoryQi <1242949407@qq.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-28 07:05:29 +00:00
Venki Korukanti 0da463e593 [SPARK-35880][SS] Track the duplicates dropped count in dedupe operator
### What changes were proposed in this pull request?

Add a metric to track the number of duplicates dropped in input in streaming deduplication operator. Also introduce a `StatefulOperatorCustomMetric` to allow stateful operators to output their own unique metrics in `StateOperatorProgress.customMetrics` in `StreamingQueryProgress`.

### Why are the changes needed?

1. Having the duplicates dropped count help monitor and debug any incorrect results issue or find reasons for state size increases in dedupe operator.
2. New API `StatefulOperatorCustomMetric` allows stateful operators to expose their own unique metrics in `StateOperatorProgress.customMetrics` in `StreamingQueryProgress`

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

Yes. For deduplication stateful operator a new metric `numDuplicatesDropped` is shown in `StateOperatorProgress` within `StreamingQueryProgress`. Example `StreamingQueryProgress` output in JSON form.

```
{
  "id" : "510be3cd-a955-4faf-8456-d97c78d39af5",
  "runId" : "c170c4cd-04cb-4a28-b054-74020e3998e1",
  ...
  ,
  "stateOperators" : [ {
    "numRowsTotal" : 1,
    "numRowsUpdated" : 1,
    "numRowsDroppedByWatermark" : 0,
    "customMetrics" : {
      "loadedMapCacheHitCount" : 0,
      "loadedMapCacheMissCount" : 0,
      "numDuplicatesDropped" : 0,
      "stateOnCurrentVersionSizeBytes" : 392
    }
  }],
  ...
}
```

### How was this patch tested?

Existing UTs for regression and added a UT.

Closes #33065 from vkorukanti/SPARK-35880.

Authored-by: Venki Korukanti <venki.korukanti@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-28 13:21:00 +09:00
Liang-Chi Hsieh b89cd8d75a [SPARK-35886][SQL] PromotePrecision should not overwrite genCode
### What changes were proposed in this pull request?

This patch fixes `PromotePrecision` where it overwrites `genCode` where subexpression elimination should happen.

### Why are the changes needed?

`PromotePrecision` overwrites `genCode` where subexpression elimination should happen. So if it is most top expression of a subexpression, it is never replaced.

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

No

### How was this patch tested?

Added test.

Closes #33103 from viirya/fix-precision.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-26 23:19:58 -07:00
zengruios b11b175148 [SPARK-35893][TESTS] Add unit test case for MySQLDialect.getCatalystType
### What changes were proposed in this pull request?
Add unit test case for MySQLDialect.getCatalystType

### Why are the changes needed?
add unit test case

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

### How was this patch tested?
Unit Test

Closes #33087 from zengruios/SPARK-35893.

Authored-by: zengruios <578395184@qq.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-26 21:43:52 -07:00
Yuming Wang def29e5075 [SPARK-35904][SQL] Collapse above RebalancePartitions
### What changes were proposed in this pull request?

1. Make `RebalancePartitions` extend `RepartitionOperation`.
2. Make `CollapseRepartition` support `RebalancePartitions`.

### Why are the changes needed?

`CollapseRepartition` can optimize `RebalancePartitions` if possible.

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

No.

### How was this patch tested?

Unit test.

Closes #33099 from wangyum/SPARK-35904.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-26 21:19:58 -07:00
Angerszhuuuu 74637a6ca7 [SPARK-35905][SQL][TESTS] Fix UT to clean up table/view in SQLQuerySuite
### What changes were proposed in this pull request?
Fix UT mistake in SQLQuerySuite

### Why are the changes needed?
Fix UT mistake in SQLQuerySuite

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

### How was this patch tested?
Existed UT

Closes #33092 from AngersZhuuuu/SPARK-33338-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-26 09:55:34 -07:00