Commit graph

8296 commits

Author SHA1 Message Date
dgd-contributor d03f71657e [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>
2021-07-13 01:28:43 +08:00
Jungtaek Lim 094300fa60 [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>
2021-07-12 09:06:50 -07:00
Chao Sun 5edbbd1711 [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>
2021-07-12 22:30:21 +08:00
Gengliang Wang 32720dd3e1 [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>
2021-07-12 10:12:30 +03:00
gengjiaan 8738682f6a [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>
2021-07-12 09:55:43 +03:00
Gengliang Wang 17ddcc9e82 [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>
2021-07-11 20:47:49 +03:00
Kent Yao f5a63322de [SPARK-36070][CORE] Log time cost info for writing rows out and committing the task
### What changes were proposed in this pull request?

We have a job that has a stage that contains about 8k tasks.  Most tasks take about 1~10min to finish but 3 of them tasks run extremely slow with similar data sizes. They take about 1 hour each to finish and also do their speculations.

The root cause is most likely the delay of the storage system. But it's not straightforward enough to find where the performance issue occurs, in the phase of shuffle read, task execution, output, commitment e.t.c..

```log
2021-07-09 03:05:17 CST SparkHadoopMapRedUtil INFO - attempt_20210709022249_0003_m_007050_37351: Committed
2021-07-09 03:05:17 CST Executor INFO - Finished task 7050.0 in stage 3.0 (TID 37351). 3311 bytes result sent to driver
2021-07-09 04:06:10 CST ShuffleBlockFetcherIterator INFO - Getting 9 non-empty blocks including 0 local blocks and 9 remote blocks
2021-07-09 04:06:10 CST TransportClientFactory INFO - Found inactive connection to
```

### Why are the changes needed?

On the spark side, we can record the time cost in logs for better bug hunting or performance tuning.

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

no

### How was this patch tested?

passing GA

Closes #33279 from yaooqinn/SPARK-36070.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-07-10 00:54:19 +08:00
ulysses-you 484b50cadf [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>
2021-07-08 22:39:53 -07:00
Takuya UESHIN 115b8a180f [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>
2021-07-09 11:30:39 +09:00
Gengliang Wang 382b66e267 [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>
2021-07-08 22:33:25 +03:00
Gengliang Wang ee945e99cc [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>
2021-07-08 21:42:50 +03:00
PengLei e071721a51 [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>
2021-07-09 01:21:38 +08:00
Wenchen Fan 2df67a1a1b [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>
2021-07-09 00:20:50 +08:00
Karen Feng 71c086eb87 [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>
2021-07-08 23:54:53 +08:00
Yuanjian Li 0621e78b5f [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>
2021-07-08 21:02:37 +09:00
Gengliang Wang 57342dfc1d [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>
2021-07-08 19:38:52 +08:00
Kousuke Saruta 39002cb995 [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>
2021-07-08 09:40:57 +03:00
Cheng Su 23943e5e40 [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>
2021-07-08 13:17:32 +09:00
Angerszhuuuu ea3333a200 [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>
2021-07-07 20:31:29 +03:00
gengjiaan 62ff2add94 [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>
2021-07-07 20:27:05 +03:00
gengjiaan cc4463e818 [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>
2021-07-07 12:41:11 +03:00
Cheng Su f3c11595ce [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>
2021-07-07 18:16:23 +09:00
allisonwang-db ca348e50a4 [SPARK-36028][SQL] Allow Project to host outer references in scalar subqueries
### What changes were proposed in this pull request?
This PR allows the `Project` node to host outer references in scalar subqueries when `decorrelateInnerQuery` is enabled. It is already supported by the new decorrelation framework and the `RewriteCorrelatedScalarSubquery` rule.

Note currently by default all correlated subqueries will be decorrelated, which is not necessarily the most optimal approach. Consider `SELECT (SELECT c1) FROM t`. This should be optimized as `SELECT c1 FROM t` instead of rewriting it as a left outer join. This will be done in a separate PR to optimize correlated scalar/lateral subqueries with OneRowRelation.

### Why are the changes needed?
To allow more types of correlated scalar subqueries.

### Does this PR introduce _any_ user-facing change?
Yes. This PR allows outer query column references in the SELECT cluase of a correlated scalar subquery. For example:
```sql
SELECT (SELECT c1) FROM t;
```
Before this change:
```
org.apache.spark.sql.AnalysisException: Expressions referencing the outer query are not supported
outside of WHERE/HAVING clauses
```

After this change:
```
+------------------+
|scalarsubquery(c1)|
+------------------+
|0                 |
|1                 |
+------------------+
```

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

Closes #33235 from allisonwang-db/spark-36028-outer-in-project.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-07-07 04:25:54 +00:00
Linhong Liu 7566db6033 [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>
2021-07-07 00:58:14 +08:00
Wenchen Fan 6b3ab8262f [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>
2021-07-06 21:17:33 +08:00
Kousuke Saruta def8bc5c96 [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>
2021-07-06 17:37:38 +08:00
Kousuke Saruta c8ff613c3c [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>
2021-07-06 11:06:56 +03:00
RoryQi e0c6b2e965 [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>
2021-07-06 14:56:12 +08:00
Yuanjian Li 9544277b0a [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>
2021-07-06 11:12:21 +09:00
Wenchen Fan 8b46e26fc6 [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>
2021-07-06 03:43:42 +08:00
Kousuke Saruta f4237aff7e [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>
2021-07-05 13:10:50 +03:00
ulysses-you 7fe4c4a9ad [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>
2021-07-05 17:10:42 +08:00
Cheng Su 044dddf288 [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>
2021-07-05 09:06:38 +00:00
Kousuke Saruta 6474226852 [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>
2021-07-05 10:35:50 +03:00
Gengliang Wang 51103cdcdd [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>
2021-07-05 12:34:21 +08:00
Wenchen Fan e6ce220690 [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>
2021-07-03 08:28:44 -07:00
Dongjoon Hyun f9f95686cb [SPARK-35996][BUILD] Setting version to 3.3.0-SNAPSHOT
### What changes were proposed in this pull request?

This PR aims to update `master` branch version to 3.3.0-SNAPSHOT.

### Why are the changes needed?

Start to prepare Apache Spark 3.3.0 and the published snapshot version should not conflict with `branch-3.2`.

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

N/A.

### How was this patch tested?

Pass the CIs.

Closes #33196 from dongjoon-hyun/SPARK-35996.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-07-02 13:47:36 -07:00
Liang-Chi Hsieh a6e00ee9d7 [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>
2021-07-02 10:58:48 -07:00
Dongjoon Hyun c55b9fd1e0 [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>
2021-07-02 09:49:49 -07:00
Wenchen Fan 0c9c8ff569 [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>
2021-07-02 16:07:31 +08:00
Yuanjian Li ca6acf0839 [SPARK-35785][SS] 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 #32933 from xuanyuanking/SPARK-35785.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-02 00:47:55 -07: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
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
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
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 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
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
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
Dongjoon Hyun f68fbae7ab [SPARK-35903][TESTS] Parameterize 'master' in TPCDSQueryBenchmark
### What changes were proposed in this pull request?

Like SPARK-8397, this PR aims to parameterize TPCDSQueryBenchmark's Spark master by reusing `spark.sql.test.master`.

### Why are the changes needed?

This is helpful for testers.

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

No. This is a test environment.

### How was this patch tested?

Manually, I checked the performance difference with TPCDS 10g data.

Closes #33098 from dongjoon-hyun/SPARK-35903.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-26 09:33:55 -07:00
Gengliang Wang 645fb59652 [SPARK-35895][SQL] Support subtracting Intervals from TimestampWithoutTZ
### What changes were proposed in this pull request?

Support the following operation:
- TimestampWithoutTZ - Year-Month interval

The following operation is actually supported in https://github.com/apache/spark/pull/33076/. This PR is to add end-to-end tests for them:
- TimestampWithoutTZ - Calendar interval
- TimestampWithoutTZ - Daytime interval

### Why are the changes needed?

Support subtracting all 3 interval types from a timestamp without time zone

### 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 #33086 from gengliangwang/subtract.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-26 13:19:00 +03:00
Anton Okolnychyi 63cd1314d2 [SPARK-35899][SQL] Utility to convert connector expressions to Catalyst
### What changes were proposed in this pull request?

This PR adds a utility to convert public connector expressions to Catalyst expressions.

Notable differences:
- Switched to `QueryCompilationErrors` from an explicit `AnalysisException`.
- Decoupled the resolving logic for v2 references into separate methods to use in other places.

### Why are the changes needed?

These changes are needed as more and more places require this logic and it is better to implement it in a single place.

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

No.

### How was this patch tested?

Existing tests.

Closes #33096 from aokolnychyi/spark-35899.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-25 18:04:07 -07:00
Jungtaek Lim 67eddf2ffc [SPARK-35894][BUILD] Introduce new style enforce to not import scala.collection.Seq/IndexedSeq
### What changes were proposed in this pull request?

This PR proposes to add a new scalastyle rule to enforce not importing `scala.collection.Seq` and `scala.collection.IndexedSeq` which conflicts with `scala.Seq` and `scala.IndexedSeq`.

The problem occurs as Scala 2.13 changed the alias of `scala.Seq` and `scala.IndexedSeq`. Before Scala 2.13, they were `scala.collection.Seq` and `scala.collection.IndexedSeq`. After Scala 2.13, they become `scala.collection.immutable.Seq` and `scala.collection.immutable.IndexedSeq`.

Please refer below doc for more details.
https://docs.scala-lang.org/overviews/core/collections-migration-213.html

### Why are the changes needed?

We have seen Seq/IndexedSeq issues on cross-compilation of Scala 2.12 / 2.13. While I'm not sure this can prevent all cases, this will prevent the simple case of breaking cross compilation.

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

No change on end user. Contributors will be restricted but shouldn't block them doing the right thing.

### How was this patch tested?

Ran scalastyle against current master (before #33084)

```
> dev/scalastyle
Scalastyle checks failed at following occurrences:
[error] /Users/Jungtaek.Lim/WorkArea/ScalaProjects/spark-apache/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala:28:0:
[error]       Don't import scala.collection.Seq and scala.collection.IndexedSeq as it may bring some problems with cross-build between Scala 2.12 and 2.13.
[error]
[error]       Please refer below page to see the details of changes around Seq.
[error]       https://docs.scala-lang.org/overviews/core/collections-migration-213.html
[error]
[error]       If you really need to use scala.collection.Seq or scala.collection.IndexedSeq, please use the fully-qualified name instead.
[error]
[error] /Users/Jungtaek.Lim/WorkArea/ScalaProjects/spark-apache/core/src/main/scala/org/apache/spark/util/Utils.scala:37:0:
[error]       Don't import scala.collection.Seq and scala.collection.IndexedSeq as it may bring some problems with cross-build between Scala 2.12 and 2.13.
[error]
[error]       Please refer below page to see the details of changes around Seq.
[error]       https://docs.scala-lang.org/overviews/core/collections-migration-213.html
[error]
[error]       If you really need to use scala.collection.Seq or scala.collection.IndexedSeq, please use the fully-qualified name instead.
[error]
[error] Total time: 15 s, completed Jun 25, 2021 9:01:32 PM
```

Closes #33085 from HeartSaVioR/SPARK-35894.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-26 09:41:16 +09:00
Gengliang Wang 9814cf8853 [SPARK-35889][SQL] Support adding TimestampWithoutTZ with Interval types
### What changes were proposed in this pull request?

Supprot the following operations:

- TimestampWithoutTZ + Calendar interval
- TimestampWithoutTZ + Year-Month interval
- TimestampWithoutTZ + Daytime interval

### Why are the changes needed?

Support basic '+' operator for timestamp without time zone 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 tests

Closes #33076 from gengliangwang/addForNewTS.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-25 19:58:42 +08:00
Yuanjian Li f2029e7442 [SPARK-35628][SS] RocksDBFileManager - load checkpoint from DFS
### What changes were proposed in this pull request?
The implementation for the load operation of RocksDBFileManager.

### Why are the changes needed?
Provide the functionality of loading all necessary files for specific checkpoint versions from DFS to the given local directory.

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

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

Closes #32767 from xuanyuanking/SPARK-35628.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-25 18:38:26 +09:00
Wenchen Fan c0cfbb1743 [SPARK-35884][SQL] EXPLAIN FORMATTED for AQE
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/29137 , which has some issues when running EXPLAIN FORMATTED
```
AdaptiveSparkPlan (13)
+- == Final Plan ==
   * HashAggregate (12)
   +- CustomShuffleReader (11)
      +- ShuffleQueryStage (10)
         +- Exchange (9)
            +- * HashAggregate (8)
               +- * Project (7)
                  +- * BroadcastHashJoin Inner BuildRight (6)
                     :- * LocalTableScan (1)
                     +- BroadcastQueryStage (5)
                        +- BroadcastExchange (4)
                           +- * Project (3)
                              +- * LocalTableScan (2)
+- == Initial Plan ==
   HashAggregate (unknown)
   +- Exchange (unknown)
      +- HashAggregate (unknown)
         +- Project (unknown)
            +- BroadcastHashJoin Inner BuildRight (unknown)
               :- Project (unknown)
               :  +- LocalTableScan (unknown)
               +- BroadcastExchange (unknown)
                  +- Project (3)
                     +- LocalTableScan (2)
```

Some nodes do not have an ID and show `unknown`. This PR fixes the issue.

### Why are the changes needed?

bug fix

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

EXPLAIN FORMATTED with AQE displays correctly.

### How was this patch tested?

new tests

Closes #33067 from cloud-fan/explain.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-25 00:18:26 -07:00
Terry Kim f1ad34558c [SPARK-35883][SQL] Migrate ALTER TABLE RENAME 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 ... RENAME 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 ... RENAME COLUMN` commands will have a consistent resolution behavior.

### How was this patch tested?

Updated existing tests.

Closes #33066 from imback82/alter_rename.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-25 05:53:56 +00:00
Adam Binford 14b1836313 [SPARK-35290][SQL] Append new nested struct fields rather than sort for unionByName with null filling
### What changes were proposed in this pull request?

This PR changes the unionByName with null filling logic to append new nested struct fields from the right side of the union to the schema versus sorting fields alphabetically. It removes the need to use UpdateField expressions, and just directly projects new nested structs from each side of the union with the correct schema. This changes the union'd schema from being alphabetically sorted previously to now "left dominant", where the fields from the left side of the union are included and then the missing ones from the right are added in the same order found originally.

### Why are the changes needed?

Certain nested structs would cause unionByName with null filling to error out due to part of the logic for rewriting the expression tree to sort the structs.

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

Yes, nested struct fields will be in a different order after unionByName with null filling than before, though shouldn't cause much effective difference.

### How was this patch tested?

Updated existing tests based on the new StructField ordering and added a new test for the case that was broken originally.

Closes #33040 from Kimahriman/union-by-name-struct-order.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-24 09:21:30 -07:00
Terry Kim 5b4816cfc8 [SPARK-34320][SQL] Migrate ALTER TABLE DROP COLUMNS commands to use UnresolvedTable to resolve the identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following `ALTER TABLE ... DROP COLUMNS` 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 ... DROP COLUMNS` commands will have a consistent resolution behavior.

### How was this patch tested?

Updated existing tests.

Closes #32854 from imback82/alter_alternative.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-24 14:59:25 +00:00
ulysses-you 1295e8876c [SPARK-35786][SQL] Add a new operator to distingush if AQE can optimize safely
### What changes were proposed in this pull request?

* Add a new repartition operator `RebalanceRepartition`.
* Support a new hint `REBALANCE`

After this patch, user can run this query:
```sql
SELECT /*+ REBALANCE(c) */ * FROM t
```

### Why are the changes needed?

Add a new hint to distingush if we can optimize it safely.

This new hint can let AQE optimize with `CustomShuffleReaderExec` safely. Currently, AQE can only coalesce shuffle partitions but can not expand shuffle partitions due to the semantics of output partitioning.
Let's say we have a query:
```sql
SELECT /*+ REPARTITION(col) */ * FROM t
```
AQE can not expand the shuffle partitions even if `col` is skewed because expanding shuffle partitions will break the hashed output paritioning of `RepartitionByExpression`. But if the query is use`REPARTITION_BY_AQE`, AQE can optimize it without considering the semantics of output partitioning.

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

Yes, a new hint.

### How was this patch tested?

Add test.

Closes #32932 from ulysses-you/SPARK-35786.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-24 09:04:38 +00:00
ulysses-you ff9ba89dcb [SPARK-35282][SQL][FOLLOWUP] Simplify condition code of shuffled hash join
### What changes were proposed in this pull request?

Simplify the condition code which is introduced by [SPARK-35282](https://issues.apache.org/jira/browse/SPARK-35282).

### Why are the changes needed?

Reduce the code size and make code more readable.

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

No

### How was this patch tested?

Pass CI

Closes #33046 from ulysses-you/simplify-shj.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-24 08:42:24 +00:00
AngersZhuuuu 7d0786f535 [SPARK-35730][SQL][TESTS] Check all day-time interval types in UDF
### What changes were proposed in this pull request?
Check all day-time interval types in UDF.

### Why are the changes needed?
New checks should improve test coverage.

### Does this PR introduce _any_ user-facing change?
Yes but `DayTimeIntervalType` has not been released yet.

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

Closes #33047 from AngersZhuuuu/SPARK-35730.

Lead-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 10:42:20 +03:00
Angerszhuuuu 490ae8f4d6 [SPARK-35777][SQL][TESTS] Check all year-month interval types in UDF
### What changes were proposed in this pull request?
Check all year-month interval types in UDF.

### Why are the changes needed?
New checks should improve test coverage.

### Does this PR introduce _any_ user-facing change?
Yes but `YearMonthIntervalType` has not been released yet.

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

Closes #32985 from AngersZhuuuu/SPARK-35777.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 08:56:08 +03:00
PengLei 61bd036cb9 [SPARK-35852][SQL] Use DateAdd instead of TimeAdd for DateType +/- INTERVAL DAY
### What changes were proposed in this pull request?
We use `DateAdd` to impl `DateType` `+`/`-`  `INTERVAL DAY`

### Why are the changes needed?
To improve the impl of `DateType` `+`/`-`  `INTERVAL DAY`
### Does this PR introduce _any_ user-facing change?
No

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

Closes #33033 from Peng-Lei/SPARK-35852.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 08:47:29 +03:00
tanel.kiis@gmail.com b3a2cebc2b [SPARK-34807][SQL] Transpose Window nodes with Project between them
### What changes were proposed in this pull request?

Extend the `TransposeWindow` rule to transpose `Window` nodes, that have `Project` between them.

### Why are the changes needed?

The analyzer will turn a `dataset.withColumn("colName", expressionWithWindowFunction)` method call to a `Project - Window - Project` chain in the logical plan. When this method is called multiple times in a row, then the projects can block the `Window` nodes from being transposed by the current `TransposeWindow` rule.

TPCDS q47 and q57 are also improved by this.

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

No

### How was this patch tested?

UT

Closes #31980 from tanelk/SPARK-34807_transpose_window.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-06-24 10:28:57 +08:00
Angerszhuuuu ad187227f1 [SPARK-35731][SQL][TESTS] Check all day-time interval types in arrow
### What changes were proposed in this pull request?
Check all day-time interval types in arrow.

### Why are the changes needed?
To improve test coverage.

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

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

Closes #33039 from AngersZhuuuu/SPARK-35731.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 23:38:41 +03:00
Kousuke Saruta 2d3fa04e90 [SPARK-35729][SQL][TESTS] Check all day-time interval types in aggregate expressions
### What changes were proposed in this pull request?

This PR adds test to check `sum` and `avg` works with all the `DayTimeIntervalType`.
This PR also moves a dataframe commonly used by tests `SPARK-34837: Support ANSI SQL intervals by the aggregate function avg` and `SPARK-34716: Support ANSI SQL intervals by the aggregate function sum` to `SQLTestData.scala`, and a little bit modifies it.

### Why are the changes needed?

To ensure the results of aggregations are what is expected.

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

No.

### How was this patch tested?

New test.

Closes #33042 from sarutak/check-interval-agg-dt.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 23:34:28 +03:00
Jungtaek Lim (HeartSaVioR) 476197791b [SPARK-34889][SS] Introduce MergingSessionsIterator merging elements directly which belong to the same session
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 MergingSessionsIterator, which enables to merge elements belong to the same session directly.

MergingSessionsIterator is a variant of SortAggregateIterator which merges the session windows based on the fact input rows are sorted by "group keys + the start time of session window". When merging windows, MergingSessionsIterator also applies aggregations on merged window, which eliminates the necessity on buffering inputs (which requires copying rows) and update the session spec for each input.

MergingSessionsIterator is quite performant compared to UpdatingSessionsIterator brought by SPARK-34888. Note that MergingSessionsIterator can only apply to the cases aggregation can be applied altogether, so there're still rooms for UpdatingSessionIterator to be used.

This issue also introduces MergingSessionsExec which is the physical node on leveraging MergingSessionsIterator to sort the input rows and aggregate rows according to the session windows.

### 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 #31987 from HeartSaVioR/SPARK-34889-SPARK-10816-PR-31570-part-2.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-23 13:04:37 -07:00
Chao Sun b8acbf6d88 [SPARK-35846][SQL] Introduce ParquetReadState to track various states while reading a Parquet column chunk
### What changes were proposed in this pull request?

Move all the bookkeeping states while scanning a Parquet column chunk into a single class `ParquetReadState`.

### Why are the changes needed?

As suggested [here](https://github.com/apache/spark/pull/32753#discussion_r655580942). To support column index in the vectorized reader path, we'll going to introduce more states to track. These are spread across different classes which make the code harder to maintain. Therefore, this proposes to move them into a single class so they can be managed better.

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

No.

### How was this patch tested?

Existing UTs.

Closes #33006 from sunchao/SPARK-35846.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-23 02:56:00 -07:00