Commit graph

11603 commits

Author SHA1 Message Date
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 be382a6285 [SPARK-36016][SQL] Support TimestampNTZType in expression ApproxCountDistinctForIntervals
### What changes were proposed in this pull request?
The current `ApproxCountDistinctForInterval`s supports `TimestampType`, but not supports timestamp without time zone yet.
This PR will add the function.

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

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

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

Closes #33243 from beliefer/SPARK-36016.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 20:22:46 +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
Angerszhuuuu 3953754f36 [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by hour to minute and unit case-sensitive issue
### What changes were proposed in this pull request?
When cast `10:10` to interval minute to second,  it can be catch by hour to minute regex, here to fix this.

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #33242 from AngersZhuuuu/SPARK-35735-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-07 12:37:19 +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
itholic 2537fe8cba [SPARK-35929][PYTHON] Support to infer nested dict as a struct when creating a DataFrame
### What changes were proposed in this pull request?

Currently, inferring nested structs is always using `MapType`.

This behavior causes an issue because it infers the schema with a value type of the first field of the struct as below:

```python
data = [{"inside_struct": {"payment": 100.5, "name": "Lee"}}]
df = spark.createDataFrame(data)
df.show(truncate=False)
+--------------------------------+
|inside_struct                   |
+--------------------------------+
|{name -> null, payment -> 100.5}|
+--------------------------------+
```

The "name" became `null`, but it should've been `"Lee"`.

In this case, we need to be able to infer the schema with a `StructType` instead of a `MapType`.

Therefore, this PR proposes adding an new configuration `spark.sql.pyspark.inferNestedDictAsStruct.enabled` to handle which type is used for inferring nested structs.
- When `spark.sql.pyspark.inferNestedDictAsStruct.enabled` is `false` (by default), inferring nested structs by `MapType`
- When `spark.sql.pyspark.inferNestedDictAsStruct.enabled` is `true`, inferring nested structs by `StructType`

### Why are the changes needed?

Because always inferring the nested structs by `MapType` doesn't work properly for some cases.

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

New configuration `spark.sql.pyspark.inferNestedDictAsStruct.enabled` is added.

### How was this patch tested?

Added an unit test

Closes #33214 from itholic/SPARK-35929.

Lead-authored-by: itholic <haejoon.lee@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-07 15:14:18 +09:00
Yuming Wang ddc5cb9051 [SPARK-35906][SQL][FOLLOWUP] Recursive remove sort if the maximum number of rows less than or equal to 1
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Unit test.

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

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-07-07 14:27:00 +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
Gengliang Wang d5d1222686 [SPARK-36025][SQL][TESTS] Reduce the run time of DateExpressionsSuite
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

Save test running time

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

No

### How was this patch tested?

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

Total reduced time: 54.3s

Closes #33229 from gengliangwang/improveTest.

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

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

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

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

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

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-07-06 13:51:06 +03: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
Angerszhuuuu 87282f04bf [SPARK-35972][SQL] When replace ExtractValue in NestedColumnAliasing we should use semanticEquals
### What changes were proposed in this pull request?
Ideally, in SQL query, nested columns should result to GetStructField with non-None name. But there are places that can create GetStructField with None name, such as UnresolvedStar.expand, Dataset encoder stuff, etc.
the current `nestedFieldToAlias` cannot catch it up and will cause job failed.

### Why are the changes needed?
Fix bug

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

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

Closes #33183 from AngersZhuuuu/SPARK-35972.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-07-06 00:09:34 -07: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
Gengliang Wang b0b9643cd7 [SPARK-35978][SQL] Support non-reserved keyword TIMESTAMP_LTZ
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33224 from gengliangwang/TIMESTAMP_LTZ.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-06 14:33:22 +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
Gengliang Wang 5f44acff3d [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ
### What changes were proposed in this pull request?

Support new keyword TIMESTAMP_NTZ, which can be used for:

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

### Why are the changes needed?

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

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33221 from gengliangwang/timstamp_ntz.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Unit test

Closes #33215 from gengliangwang/tsLiteral.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-06 00:54:58 +08:00
gengjiaan c605ba2d46 [SPARK-35664][SQL][FOLLOWUP] Fix incorrect comment for TimestampNTZType
### What changes were proposed in this pull request?
This PR fix the incorrect comment for `TimestampNTZType`.

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

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

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

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

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-07-05 18:48:00 +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
Gengliang Wang a643076d4e [SPARK-35975][SQL] New configuration spark.sql.timestampType for the default timestamp type
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

No, it's a new feature.

### How was this patch tested?

Unit test

Closes #33176 from gengliangwang/newTsTypeConf.

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

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

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

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

after fix we are getting the result

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

```

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

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

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

Closes #32972 from SaurabhChawla100/SPARK-35756.

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

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

### Why are the changes needed?

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

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit tests

Closes #33173 from gengliangwang/reviseTypeName.

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

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

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

### How was this patch tested?
not yet

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

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

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

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

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

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

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

Closes #33169 from yaooqinn/SPARK-35966.

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

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

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

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

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

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

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

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

### How was this patch tested?
Unit tests

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

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

Rename TimestampWithoutTZType to TimestampNTZType

### Why are the changes needed?

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

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

No, the new timestamp type is not released yet.

### How was this patch tested?

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

Closes #33167 from gengliangwang/rename.

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

Remove dead code in `OptimizeLocalShuffleReader`.

### Why are the changes needed?

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

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

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

No

### How was this patch tested?

Pass CI

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

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

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

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

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

Closes #32958 from beliefer/SPARK-35065.

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

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

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

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

a new test

Closes #33158 from cloud-fan/bug.

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

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

### Why are the changes needed?

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

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

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

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

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

### How was this patch tested?

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

Closes #32753 from sunchao/SPARK-34859.

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

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

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

### Why are the changes needed?

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

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

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

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

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

Yes, a new config

### How was this patch tested?

Add test

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

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

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

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

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

### How was this patch tested?

Unit tests

Closes #33156 from gengliangwang/dateField.

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

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

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

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

Closes #32943 from AngersZhuuuu/SPARK-35735.

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

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

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

### Why are the changes needed?

Support basic operations for the new timestamp type.

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

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

### How was this patch tested?

Unit test

Closes #33136 from gengliangwang/field.

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Unit tests, as well as local tests with PyODBC.

Closes #32850 from karenfeng/SPARK-34920.

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

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

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

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

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

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

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

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

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

Closes #33143 from MaxGekk/revert-make_interval.

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

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

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

### Why are the changes needed?

Fix a bug.

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

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

### How was this patch tested?

Unit test

Closes #33138 from gengliangwang/fixGetDateField.

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

This patch refactors the evaluation of subexpressions.

There are two changes:

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing tests.

Closes #32980 from viirya/subexpr-eval.

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Existing tests for regressions. Added new UTs.

Closes #33091 from vkorukanti/SPARK-35896.

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

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

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

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

Closes #32928 from xuanyuanking/SPARK-35784.

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

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

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

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #33100 from wangyum/SPARK-35906.

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

This PR aims to upgrade ASM to 9.1

### Why are the changes needed?

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

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

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

No.

### How was this patch tested?

Pass the CIs.

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

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

Skip empty partitions in `ShufflePartitionsUtil.coalescePartitionsWithSkew`.

### Why are the changes needed?

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

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

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

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

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

No, not release yet.

### How was this patch tested?

Add test.

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing UT

Closes #33129 from gengliangwang/allTimestampTypes.

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

Support the following operations:

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

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

### Why are the changes needed?

Support basic subtraction among Date/Timestamp/TimestampWithoutTZ.

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

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

### How was this patch tested?

Unit tests

Closes #33115 from gengliangwang/subtractTimestampWithoutTz.

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

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

### Why are the changes needed?

These changes are needed to simply the utility API.

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

No.

### How was this patch tested?

Existing tests.

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

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

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

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

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

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

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

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

### Why are the changes needed?

This is apparently a bug.

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

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

### How was this patch tested?

New tests.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Updated existing tests.

Closes #33113 from imback82/alter_change_column.

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

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

### Why are the changes needed?

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

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

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

No

### How was this patch tested?

Pass CI

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

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

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

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

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

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

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

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

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

This PR fixes these issues.

### Why are the changes needed?

Both bugs cause an exception to be thrown.

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

No

### How was this patch tested?

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

Closes #33108 from tomvanbussel/SPARK-35898.

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

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

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

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

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

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

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

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

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

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

Closes #32916 from dgd-contributor/SPARK-35064_catalyst_group_error.

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

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

### Does this PR introduce _any_ user-facing change?
Yes
Before this change, the user will see below output for  `show tblproperties test_view`
```
....
p1 v1
p2 v2
view.catalogAndNamespace.numParts	2
view.catalogAndNamespace.part.0	spark_catalog
view.catalogAndNamespace.part.1	default
view.query.out.col.0	c1
view.query.out.numCols	1
view.referredTempFunctionsNames	[]
view.referredTempViewNames	[]
...
```
After this change, the internal properties will be hidden.
```
....
p1 v1
p2 v2
...
```
### How was this patch tested?
existing UT

Closes #33016 from jerqi/hide_show_tblproperties.

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Existing UTs for regression and added a UT.

Closes #33065 from vkorukanti/SPARK-35880.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Added test.

Closes #33103 from viirya/fix-precision.

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

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

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

### How was this patch tested?
Unit Test

Closes #33087 from zengruios/SPARK-35893.

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

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

### Why are the changes needed?

`CollapseRepartition` can optimize `RebalancePartitions` if possible.

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

No.

### How was this patch tested?

Unit test.

Closes #33099 from wangyum/SPARK-35904.

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

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

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

### How was this patch tested?
Existed UT

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

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-26 09:55:34 -07:00
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
Kousuke Saruta 156b9b5d14 [SPARK-35736][SPARK-35774][SQL][FOLLOWUP] Prohibit to specify the same units for FROM and TO with unit-to-unit interval syntax
### What changes were proposed in this pull request?

This PR change the behavior of unit-to-unit interval syntax to prohibit the case that the same units are specified for FROM and TO.

### Why are the changes needed?

For ANSI compliance.

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

No.

### How was this patch tested?

New test.

Closes #33057 from sarutak/prohibit-unit-pattern.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 23:13:31 +03: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
Angerszhuuuu de35675c61 [SPARK-35871][SQL] Literal.create(value, dataType) should support fields
### What changes were proposed in this pull request?
Current Literal.create(data, dataType) for Period to YearMonthIntervalType and Duration to DayTimeIntervalType is not correct.

if data type is Period/Duration, it will create converter of default YearMonthIntervalType/DayTimeIntervalType,  then the result is not correct, this pr fix this bug.

### Why are the changes needed?
Fix  bug when use Literal.create()

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

### How was this patch tested?
Added UT

Closes #33056 from AngersZhuuuu/SPARK-35871.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 17:36:48 +03:00
Max Gekk d40a1a2552 Revert "[SPARK-35728][SQL][TESTS] Check multiply/divide of day-time intervals of any fields by numeric"
### What changes were proposed in this pull request?
Revert 8a1995f936

### Why are the changes needed?
The merged test doesn't check different interval fields, actually. Need to apply this https://github.com/apache/spark/pull/33056 first of all.

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

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

Closes #33060 from MaxGekk/revert-Peng-Lei-SPARK-35728.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 14:36:07 +03:00
Max Gekk 345d3db83d Revert "[SPARK-35778][SQL][TESTS] Check multiply/divide of year month interval of any fields by numeric"
### What changes were proposed in this pull request?
Revert 3904c0edba

### Why are the changes needed?
The merged test doesn't check different interval fields, actually. Need to apply this https://github.com/apache/spark/pull/33056 first of all.

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

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

Closes #33059 from MaxGekk/revert-Peng-Lei-SPARK-35778.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 14:34:42 +03:00
PengLei 3904c0edba [SPARK-35778][SQL][TESTS] Check multiply/divide of year month interval of any fields by numeric
### What changes were proposed in this pull request?
Check multiply/divide of year-month intervals of any fields by numeric.

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

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

### How was this patch tested?
Expanded existed test cases.

Closes #33051 from Peng-Lei/SPARK-35778.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 12:25:06 +03:00
PengLei 8a1995f936 [SPARK-35728][SQL][TESTS] Check multiply/divide of day-time intervals of any fields by numeric
### What changes were proposed in this pull request?
1. The testcase is just cover the DayTimeIntervalType() */ numeric
2. Add testcase for following intervals */ numeric:
   INTERVAL DAY
   INTERVAL DAY TO HOUR
   INTERVAL DAY TO MINUTE
   INTERVAL HOUR
   INTERVAL HOUR TO MINUTE
   INTERVAL HOUR TO SECOND
   INTERVAL MINUTE
   INTERVAL MINUTE TO SECOND
   INTERVAL SECOND

### Why are the changes needed?
Add testcase coverage.

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

### How was this patch tested?
existed testcase

Closes #33014 from Peng-Lei/SPARK-35728.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-24 12:11:47 +03: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
dgd-contributor 5b9c5c126f [SPARK-35841][SQL] Casting string to decimal type doesn't work if the…
… sum of the digits is greater than 38
### What changes were proposed in this pull request?
Since Spark 3.1.1, NULL is returned when casting a string with many decimal places to a decimal type. If the sum of the digits before and after the decimal point is less than 39, a value is returned. From 39 digits, however, NULL is returned.
This worked until Spark 3.0.X.

Code to reproduce:

A string with 2 decimal places in front of the decimal point and 37 decimal places after the decimal point returns null

```
val data = Seq(
      "28.9259999999999983799625624669715762138",
      "28.925999999999998379962562466971576213",
      "2.9259999999999983799625624669715762138"
      )
val df = data.toDF("num")
df.withColumn("numConverted", col("num").cast("decimal(38, 5)")).show()
```

before this pull request, the result is
+----------------------+---------------+
|                 num          |numConverted|
+----------------------+---------------+
|28.92599999999999...|                  null|
|28.92599999999999...|         28.92600|
|2.925999999999998...|           2.92600|
+----------------------+---------------+

the correct result should be
+----------------------+---------------+
|                 num          |numConverted|
+----------------------+---------------+
|28.92599999999999...|         28.92600|
|28.92599999999999...|         28.92600|
|2.925999999999998...|           2.92600|
+----------------------+---------------+

The problem occur since https://issues.apache.org/jira/browse/SPARK-32706, it because the fast fail is checking precision length, which should only check the whole number part length of the input value, not the precision length

### Why are the changes needed?
correctness

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

### How was this patch tested?
test added

Closes #33011 from dgd-contributor/SPARK-35841_castStringToDecimalTypeError.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-24 16:44:58 +08: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 5e77ca8071 [SPARK-35768][SQL] Take into account year-month interval fields in cast
### What changes were proposed in this pull request?
Support take into account year-month interval field in cast

##### Rule cast to target YearMonthIntervalType

|  string  | demo | strict target type  |   months |
|---|---|---|---|
|  [+\|-]y-m | 1-1  | YearMonthIntervalType(YEAR. MONTH) | 13  |
| [+\|-]y| 1 | YearMonthIntervalType(YEAR. YEAR) | 12  |
| [+\|-]m | 1 | YearMonthIntervalType(MONTH. MONTH) | 1  |
|  INTERVAL [+\|-]'[+\|-]y-m' YEAR TO MONTH | interval '1-1' year to month | YearMonthIntervalType(YEAR. MONTH) | 13  |
|  INTERVAL [+\|-]'[+\|-]m' MONTH | interval '1' month | YearMonthIntervalType(MONTH. MONTH) |  1 |
|  INTERVAL [+\|-]'[+\|-]y' YEAR | interval '1' year | YearMonthIntervalType(YEAR.YEAR) | 12 |

### Why are the changes needed?
Support take into account year-month interval field in cast

### Does this PR introduce _any_ user-facing change?
user can use `cast(str, YearMonthInterval(YEAR, YEAR))` etc

### How was this patch tested?
Added UT

Closes #32940 from AngersZhuuuu/SPARK-35768.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-24 07:48:47 +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
Vinod KC 4dabba8f76 [SPARK-35747][CORE] Avoid printing full Exception stack trace, if Hbase/Kafka/Hive services are not running in a secure cluster
### What changes were proposed in this pull request?
In a secure Yarn cluster, even though HBase or Kafka, or Hive services are not used in the user application, yarn client unnecessarily trying to generate  Delegations token from these services. This will add additional delays while submitting spark application in a yarn cluster

 Also during HBase delegation token generation step in the application submit stage,  HBaseDelegationTokenProvider prints a full Exception Stack trace and it causes a noisy warning.
 Apart from printing exception stack trace, Application submission taking more time as it retries connection to HBase master multiple times before it gives up. So, if HBase is not used in the user Applications, it is better to suggest User disable HBase Delegation Token generation.

 This PR aims to avoid printing full Exception Stack by just printing just Exception name and also add a suggestion message to disable `Delegation Token generation` if service is not used in the Spark Application.

 eg: `If HBase is not used, set spark.security.credentials.hbase.enabled to false`

### Why are the changes needed?

To avoid printing full Exception stack trace in WARN log
#### Before the fix
----------------
```
spark-shell --master yarn
.......
.......
21/06/12 14:29:41 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase
java.lang.reflect.InvocationTargetException
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at org.apache.spark.deploy.security.HBaseDelegationTokenProvider.obtainDelegationTokensWithHBaseConn(HBaseDelegationT
okenProvider.scala:93)
        at org.apache.spark.deploy.security.HBaseDelegationTokenProvider.obtainDelegationTokens(HBaseDelegationTokenProvider.
scala:60)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager$$anonfun$6.apply(HadoopDelegationTokenManager.scala:
166)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager$$anonfun$6.apply(HadoopDelegationTokenManager.scala:
164)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
        at scala.collection.Iterator$class.foreach(Iterator.scala:891)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
        at scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
        at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
        at scala.collection.AbstractTraversable.flatMap(Traversable.scala:104)
        at org.apache.spark.deploy.security.HadoopDelegationTokenManager.obtainDelegationTokens(HadoopDelegationTokenManager.
scala:164)
```

#### After  the fix
------------
```
 spark-shell --master yarn

Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
21/06/13 02:10:02 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase due to  java.lang.reflect.InvocationTargetException Retrying to fetch HBase security token with hbase connection parameter.
21/06/13 02:10:40 WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase java.lang.reflect.InvocationTargetException. If HBase is not used, set spark.security.credentials.hbase.enabled to false
21/06/13 02:10:47 WARN cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Attempted to request executors before the AM has registered!
```
### Does this PR introduce _any_ user-facing change?

Yes, in the log, it avoids printing full Exception stack trace.
Instread prints this.
**WARN security.HBaseDelegationTokenProvider: Failed to get token from service hbase java.lang.reflect.InvocationTargetException. If HBase is not used, set spark.security.credentials.hbase.enabled to false**

### How was this patch tested?

Tested manually as it can be verified only in a secure cluster

Closes #32894 from vinodkc/br_fix_Hbase_DT_Exception_stack_printing.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-23 23:12:02 -07: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
Angerszhuuuu 077cf2acdb [SPARK-35733][SQL][TESTS] Check all day-time interval types in HiveInspectors tests
### What changes were proposed in this pull request?
Check all day-time interval types in HiveInspectors tests.

### Why are the changes needed?
New tests should improve test coverage for day-time interval types.

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

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

Closes #33036 from AngersZhuuuu/SPARK-35733.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 19:20:51 +03: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
Gengliang Wang 6f51e37eb5 [SPARK-35857][SQL] The ANSI flag of Cast should be kept after being copied
### What changes were proposed in this pull request?

Make the ANSI flag part of expression `Cast`'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 expression `Cast`

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

Yes, the `Cast` 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 #33027 from gengliangwang/ansiFlagInCast.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-23 16:52:33 +08:00
Angerszhuuuu 758b423a31 [SPARK-35860][SQL] Support UpCast between different field of YearMonthIntervalType/DayTimeIntervalType
### What changes were proposed in this pull request?
Support UpCast between different field of YearMonthIntervalType/DayTimeIntervalType

### Why are the changes needed?
Since in our encoder we handle Period/Duration as default  YearMonthIntervalType/DayTimeIntervalType, when we use udf to handle this type, it will upcast all type of YearMonthIntervalType/DayTimeIntervalType to default YearMonthIntervalType/DayTimeIntervalType, so we need to support this.

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

### How was this patch tested?
Added Ut

Closes #33035 from AngersZhuuuu/SPARK-35860.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 11:32:13 +03:00
Angerszhuuuu 7c1a9dd3f5 [SPARK-35776][SQL][TESTS] Check all year-month interval types in arrow
### What changes were proposed in this pull request?
Add tests to check that all year-month interval types are supported in (de-)serialization from/to Arrow format.

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

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

### How was this patch tested?
added ut

Closes #32993 from AngersZhuuuu/SPARK-35776.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 10:59:50 +03:00
Peter Toth 79e3d0d98f [SPARK-35855][SQL] Unify reuse map data structures in non-AQE and AQE rules
### What changes were proposed in this pull request?
This PR unifies reuse map data structures in non-AQE and AQE rules to a simple `Map[<canonicalized plan>, <plan>]` based on the discussion here: https://github.com/apache/spark/pull/28885#discussion_r655073897

### Why are the changes needed?
The proposed `Map[<canonicalized plan>, <plan>]` is simpler than the currently used `Map[<schema>, ArrayBuffer[<plan>]]` in `ReuseMap`/`ReuseExchangeAndSubquery` (non-AQE) and consistent with the `ReuseAdaptiveSubquery` (AQE) subquery reuse rule.

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

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

Closes #33021 from peter-toth/SPARK-35855-unify-reuse-map-data-structures.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-23 07:20:47 +00:00
Wenchen Fan 20edfdd39a [SPARK-35845][SQL] OuterReference resolution should reject ambiguous column names
### What changes were proposed in this pull request?

The current OuterReference resolution is a bit weird: when the outer plan has more than one child, it resolves OuterReference from the output of each child, one by one, left to right.

This is incorrect in the case of join, as the column name can be ambiguous if both left and right sides output this column.

This PR fixes this bug by resolving OuterReference with `outerPlan.resolveChildren`, instead of something like `outerPlan.children.foreach(_.resolve(...))`

### Why are the changes needed?

bug fix

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

The problem only occurs in join, and join condition doesn't support correlated subquery yet. So this PR only improves the error message. Before this PR, people see
```
java.lang.UnsupportedOperationException
Cannot generate code for expression: outer(t1a#291)
```

### How was this patch tested?

a new test

Closes #33004 from cloud-fan/outer-ref.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-23 14:32:34 +08:00
Angerszhuuuu df55945804 [SPARK-35772][SQL][TESTS] Check all year-month interval types in HiveInspectors tests
### What changes were proposed in this pull request?
Check all year-month interval types in HiveInspectors tests.

### 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 #32970 from AngersZhuuuu/SPARK-35772.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 08:54:07 +03:00
Kousuke Saruta 4416b4b8ba [SPARK-35734][SQL][FOLLOWUP] IntervalUtils.toDayTimeIntervalString should consider the case a day-time type is casted as another day-time type
### What changes were proposed in this pull request?

This PR fixes an issue that `IntervalUtils.toDayTimeIntervalString` doesn't consider the case that a day-time interval type is casted as another day-time interval type.
if data of `interval day to second` is casted as `interval hour to second`, the value of the day is multiplied by 24 and added to the value of hour. For example, `INTERVAL '1 2' DAY TO HOUR` will be `INTERVAL '26' HOUR` if it's casted.
If this behavior is intended, it should be stringified as `INTERVAL '26' HOUR` but currently, it will be `INTERVAL '2' HOUR`

### Why are the changes needed?

t's a bug if the behavior of cast is intended.

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

No, because this feature is not released yet.

### How was this patch tested?

Modified the tests added in SPARK-35734 (#32891)

Closes #33031 from sarutak/fix-toDayTimeIntervalString.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-23 08:00:35 +03:00
Gengliang Wang 960a7e5fce [SPARK-35856][SQL][TESTS] Move new interval type test cases from CastSuite to CastBaseSuite
### What changes were proposed in this pull request?

There are a few test cases that are supposed to be in CastSuiteBase instead of CastSuite:

- SPARK-35112: Cast string to day-time interval
- SPARK-35111: Cast string to year-month interval
- SPARK-35820: Support cast DayTimeIntervalType in different fields
- SPARK-35819: Support cast YearMonthIntervalType in different fields

This PR is to move them to CastSuiteBase. Also, it adds comments for the scope of CastSuiteBase/CastSuite/AnsiCastSuiteBase.
### Why are the changes needed?

Increase test coverage so that we can test the casting under ANSI mode.

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

No

### How was this patch tested?

Existing UT

Closes #33022 from gengliangwang/moveTest.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-23 11:20:50 +08:00
Wenchen Fan a87ee5d8b9 [SPARK-35695][SQL][FOLLOWUP] Use AQE helper to simplify the code in CollectMetricsExec
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/32862 , to simplify the code with AQE helper.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #33026 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-23 09:54:12 +09:00
Wenchen Fan 7a21e9c48f [SPARK-35858][SQL] SparkPlan.makeCopy should not set the active session
### What changes were proposed in this pull request?

We introduced `SparkSession.withActive` a while ago, and we use it when we need to run some code with a certain SparkSession as the active session.

Somehow we missed `SparkPlan.makeCopy`, which sets active session directly. This PR proposes to call `SparkSession.withActive` there.

### Why are the changes needed?

make sure we don't change the active session unexpectedly.

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

No. `makeCopy` is an internal function and I can't find a real case that this can change the active session. Mostly in an upper level, there is already a `SparkSession.withActive`, like `QueryExecution.executePhase`

### How was this patch tested?

existing tests

Closes #33029 from cloud-fan/minor1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-23 09:50:59 +09:00
Wenchen Fan a2c1a55b1f [SPARK-35700][SQL][FOLLOWUP] Read schema from ORC files should strip CHAR/VARCHAR types
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/33001 , to provide a more direct fix.

The regression in 3.1 was caused by the fact that we changed the parser and allow the parser to return CHAR/VARCHAR type. We should have replaced CHAR/VARCHAR with STRING before the data type flows into the query engine, however, `OrcUtils` is missed.

When reading ORC files, at the task side we will read the real schema from ORC file metadata, then apply filter pushdown. For some reason, the implementation turns ORC schema to Spark schema before filter pushdown, and this step does not strip CHAR/VARCHAR. Note, for Parquet we use the Parquet schema directly in filter pushdown, and do not this have problem.

This PR proposes to replace the CHAR/VARCHAR with STRING when turning ORC schema to Spark schema.

### Why are the changes needed?

a more directly bug fix

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

no

### How was this patch tested?

existing tests

Closes #33030 from cloud-fan/help.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 13:50:49 -07:00
Li Zhang dfd7b026dc [SPARK-35800][SS] Improving GroupState testability by introducing TestGroupState
### What changes were proposed in this pull request?
Proposed changes in this pull request:

1. Introducing the `TestGroupState` interface which is inherited from `GroupState` so that testing related getters can be exposed in a controlled manner
2. Changing `GroupStateImpl` to inherit from `TestGroupState` interface, instead of directly from `GroupState`
3. Implementing `TestGroupState` object with `create()` method to forward inputs to the private `GroupStateImpl` constructor
4. User input validations have been added into `GroupStateImpl`'s `createForStreaming()` method to prevent users from creating invalid GroupState objects.
5. Replacing existing `GroupStateImpl` usages in sql pkg internal unit tests with the newly added `TestGroupState` to give user best practice about `TestGroupState` usage.

With the changes in this PR, the class hierarchy is changed from `GroupStateImpl` -> `GroupState` to `GroupStateImpl` -> `TestGroupState` -> `GroupState` (-> means inherits from)

### Why are the changes needed?
The internal `GroupStateImpl` implementation for the `GroupState` interface has no public constructors accessible outside of the sql pkg. However, the user-provided state transition function for `[map|flatMap]GroupsWithState` requires a `GroupState` object as the prevState input.

Currently, users are calling the Structured Streaming engine in their unit tests in order to instantiate such `GroupState` instances, which makes UTs cumbersome.

The proposed `TestGroupState` interface is to give users controlled access to the `GroupStateImpl` internal implementation to largely improve testability of Structured Streaming state transition functions.

**Usage Example**
```
import org.apache.spark.sql.streaming.TestGroupState

test(“Structured Streaming state update function”) {
  var prevState = TestGroupState.create[UserStatus](
    optionalState = Optional.empty[UserStatus],
    timeoutConf = EventTimeTimeout,
    batchProcessingTimeMs = 1L,
    eventTimeWatermarkMs = Optional.of(1L),
    hasTimedOut = false)

  val userId: String = ...
  val actions: Iterator[UserAction] = ...

  assert(!prevState.hasUpdated)

  updateState(userId, actions, prevState)

  assert(prevState.hasUpdated)
}
```

### Does this PR introduce _any_ user-facing change?
Yes, the `TestGroupState` interface and its corresponding `create()` factory function in its companion object are introduced in this pull request for users to use in unit tests.

### How was this patch tested?
- New unit tests are added
- Existing GroupState unit tests are updated

Closes #32938 from lizhangdatabricks/improve-group-state-testability.

Authored-by: Li Zhang <li.zhang@databricks.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2021-06-22 15:04:01 -04:00
Gengliang Wang ce53b7199d [SPARK-35854][SQL] Improve the error message of to_timestamp_ntz with invalid format pattern
### What changes were proposed in this pull request?

When SQL function `to_timestamp_ntz` has invalid format pattern input, throw a runtime exception with hints for the valid patterns, instead of throwing an upgrade exception with suggestions to use legacy formatters.

### Why are the changes needed?

As discussed in https://github.com/apache/spark/pull/32995/files#r655148980, there is an error message saying
"You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0"

This is not true for function to_timestamp_ntz, which only uses the Iso8601TimestampFormatter and added since Spark 3.2. We should improve it.

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

No, the new SQL function is not released yet.

### How was this patch tested?

Unit test

Closes #33019 from gengliangwang/improveError.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-22 23:45:54 +08:00
Lei Peng bc61b62a55 [SPARK-35727][SQL] Return INTERVAL DAY from dates subtraction
What changes were proposed in this pull request?

1. Change the return value type from DayTimeIntervalType(DAY, SECOND) to DayTimeIntervalType(DAY, DAY) of SubtractDates.

Why are the changes needed?
https://issues.apache.org/jira/browse/SPARK-35727

Does this PR introduce any user-facing change?
no

How was this patch tested?
existed ut test

Closes #32999 from Peng-Lei/SPARK-35727.

Lead-authored-by: Lei Peng <peng.8lei@gmail.com>
Co-authored-by: PengLei <18066542445@189.cn>
Co-authored-by: Peng-Lei <peng.8lei@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-22 13:43:25 +00:00
YangJie 6c05459600 [SPARK-35838][BUILD][TESTS] Ensure all modules can be maven test independently in Scala 2.13
### What changes were proposed in this pull request?
Similar to SPARK-35532, the main change of this pr is add `scala-2.13` profile to external/kafka-0-10-sql/pom.xml, external/avro/pom.xml and sql/hive-thriftserver/pom.xml,  the `scala-2.13` profile include dependency on `scala-parallel-collections_2.13`, then all(34) spark modules can maven test independently.

### Why are the changes needed?
Ensure alll(34) spark modules can be maven test independently in Scala 2.13

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

### How was this patch tested?
- Pass the GitHub Action Scala 2.13 job
- Manual test:

1. Execute
```
dev/change-scala-version.sh 2.13

mvn clean install -DskipTests -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13
```

2. maven test `external/kafka-0-10-sql` module
```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl external/kafka-0-10-sql
```

**before**

```
Discovery starting.
Discovery completed in 857 milliseconds.
Run starting. Expected test count is: 464
...
KafkaRelationSuiteV2:
- explicit earliest to latest offsets
- default starting and ending offsets
- explicit offsets
- default starting and ending offsets with headers
- timestamp provided for starting and ending
- timestamp provided for starting, offset provided for ending
- timestamp provided for ending, offset provided for starting
- timestamp provided for starting, ending not provided
- timestamp provided for ending, starting not provided
- global timestamp provided for starting and ending
- no matched offset for timestamp - startingOffsets
- preferences on offset related options
- no matched offset for timestamp - endingOffsets
*** RUN ABORTED ***
  java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
  ...
  Cause: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
  at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  ...
```

**After**

```
Run completed in 33 minutes, 51 seconds.
Total number of tests run: 464
Suites: completed 31, aborted 0
Tests: succeeded 464, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

3. maven test `external/avro` module

```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl external/avro
```

**before**

```
Discovery starting.
Discovery completed in 2 seconds, 765 milliseconds.
Run starting. Expected test count is: 255
AvroReadSchemaSuite:
- append column at the end
- hide column at the end
- append column into middle
- hide column in the middle
- add a nested column at the end of the leaf struct column
- add a nested column in the middle of the leaf struct column
- add a nested column at the end of the middle struct column
- add a nested column in the middle of the middle struct column
- hide a nested column at the end of the leaf struct column
- hide a nested column in the middle of the leaf struct column
- hide a nested column at the end of the middle struct column
- hide a nested column in the middle of the middle struct column
*** RUN ABORTED ***
  java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
  ...
  Cause: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
  at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
  at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
  at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
  at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
  at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
  at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
  at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
  at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
  ...
```

**After**

```
Run completed in 1 minute, 42 seconds.
Total number of tests run: 255
Suites: completed 12, aborted 0
Tests: succeeded 255, failed 0, canceled 0, ignored 2, pending 0
All tests passed.
```

4.  maven test `sql/hive-thriftserver` module

```
mvn test -Phadoop-3.2 -Phive-2.3 -Phadoop-cloud -Pmesos -Pyarn -Pkinesis-asl -Phive-thriftserver -Pspark-ganglia-lgpl -Pkubernetes -Phive -Pscala-2.13 -pl sql/hive-thriftserver
```

**before**

```
- union.sql *** FAILED ***
  "1  a
  1 a
  2 b
  2 b" did not contain "Exception" Exception did not match for query #2
  SELECT *
  FROM   (SELECT * FROM t1
          UNION ALL
          SELECT * FROM t1), expected: 1  a
  1 a
  2 b
  2 b, but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
    at org.apache.spark.sql.hive.thriftserver.HiveThriftServerErrors$.runningQueryError(HiveThriftServerErrors.scala:38)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:324)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.$anonfun$run$2(SparkExecuteStatementOperation.scala:229)
    at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
    at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties(SparkOperation.scala:79)
    at org.apache.spark.sql.hive.thriftserver.SparkOperation.withLocalProperties$(SparkOperation.scala:63)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withLocalProperties(SparkExecuteStatementOperation.scala:43)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:229)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2$$anon$3.run(SparkExecuteStatementOperation.scala:224)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:422)
    at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$2.run(SparkExecuteStatementOperation.scala:238)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)
  Caused by: java.lang.NoClassDefFoundError: scala/collection/parallel/TaskSupport
    at org.apache.spark.SparkContext.$anonfun$union$1(SparkContext.scala:1411)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
    at org.apache.spark.SparkContext.withScope(SparkContext.scala:788)
    at org.apache.spark.SparkContext.union(SparkContext.scala:1405)
    at org.apache.spark.sql.execution.UnionExec.doExecute(basicPhysicalOperators.scala:697)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:182)
    at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:220)
    at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
    at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:217)
    at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:178)
    at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:323)
    at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:389)
    at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3719)
    at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:2987)
    at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3710)
    at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
    at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
    at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
    at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:774)
    at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
    at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3708)
    at org.apache.spark.sql.Dataset.collect(Dataset.scala:2987)
    at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:299)
    ... 16 more
  Caused by: java.lang.ClassNotFoundException: scala.collection.parallel.TaskSupport
    at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
    at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
    at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
    at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
    ... 40 more (ThriftServerQueryTestSuite.scala:209)
```

**After**

```
Run completed in 29 minutes, 17 seconds.
Total number of tests run: 535
Suites: completed 20, aborted 0
Tests: succeeded 535, failed 0, canceled 0, ignored 17, pending 0
All tests passed.
```

Closes #32994 from LuciferYang/SPARK-35838.

Authored-by: YangJie <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-22 06:31:24 -07:00
Angerszhuuuu 5a510cf578 [SPARK-35726][SPARK-35769][SQL][FOLLOWUP] Call periodToMonths and durationToMicros in HiveResult should add endField
### What changes were proposed in this pull request?
When we call periodToMonths and durationToMicros  with certain type field, we should pass endField parameter.

### Why are the changes needed?
When we call periodToMonths and durationToMicros  with certain type field, we should pass endField parameter.

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

### How was this patch tested?
Existed UT

Closes #32984 from AngersZhuuuu/SPARK-35726-35769.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-22 11:15:35 +03:00
gengjiaan 43cd6ca687 [SPARK-35378][SQL][FOLLOWUP] isLocal should consider CommandResult
### What changes were proposed in this pull request?
#32513 added the case class `CommandResult` so as we can eagerly execute command locally. But we forgot to update
`isLocal` of `Dataset`.

### Why are the changes needed?
`Dataset.isLocal` should consider `CommandResult`.

### Does this PR introduce _any_ user-facing change?
Yes. If the SQL plan is `CommandResult`, `Dataset.isLocal` must return true.

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

Closes #32963 from beliefer/SPARK-35378-followup2.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-22 07:39:54 +00:00
Venki Korukanti d4d11cfbfb [SPARK-35799][SS] Fix the allUpdatesTimeMs metric measuring in FlatMapGroupsWithStateExec
### What changes were proposed in this pull request?

Fix how we measure the metric `allUpdatesTimeMs` in `FlatMapGroupsWithStateExec` similar to other streaming stateful operators.

### Why are the changes needed?

Metric `allUpdatesTimeMs` meant to capture the start to end walltime of the operator `FlatMapGroupsWithStateExec`, but currently it just [captures](79362c4efc/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala (L121)) the iterator creation time.

Fix it to measure similar to how other stateful operators measure. Example one [here](79362c4efc/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala (L406)). This measurement is not perfect due to the nature of the lazy iterator and also includes the time the consumer operator spent in processing the current operator output, but it should give a good signal when comparing the metric in one microbatch to the metric in another microbatch.

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

No

### How was this patch tested?

Existing UTs for regression. Due to the nature of metric type (time), it is hard to write a UT, but have manually verified.

Closes #32952 from vkorukanti/SPARK-35799.

Authored-by: Venki Korukanti <venki.korukanti@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-22 13:57:21 +09:00
Kent Yao 9f734978d9 [SPARK-35700][SQL] Read char/varchar orc table with created and written by external systems
### What changes were proposed in this pull request?

The char/varchar type should be mapped to orc's string type too, see https://orc.apache.org/docs/types.html

### Why are the changes needed?

fix a regression

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

no

### How was this patch tested?

new tests

Closes #33001 from yaooqinn/SPARK-35700.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-21 19:20:55 -07:00
Gengliang Wang 2bdd9fe5e3 [SPARK-35839][SQL] New SQL function: to_timestamp_ntz
### What changes were proposed in this pull request?

Implement new SQL function: `to_timestamp_ntz`.
The syntax is similar to the built-in function `to_timestamp`:
```
to_timestamp_ntz ( <date_expr> )

to_timestamp_ntz ( <timestamp_expr> )

to_timestamp_ntz ( <string_expr> [ , <format> ] )
```

The naming is from snowflake: https://docs.snowflake.com/en/sql-reference/functions/to_timestamp.html

### Why are the changes needed?

Adds a new SQL function to create a literal/column of timestamp without time zone.
It's convenient for both end-users and developers.

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

Yes, a new SQL function `to_timestamp_ntz`.

### How was this patch tested?

Unit tests

Closes #32995 from gengliangwang/toTimestampNtz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-22 09:50:48 +08:00
Kousuke Saruta 2c91672259 [SPARK-35775][SQL][TESTS] Check all year-month 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 `YearMonthInterval` types.

### 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 #32988 from sarutak/check-interval-agg-ym.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 16:47:29 +03:00
tanel.kiis@gmail.com f80be4187e [SPARK-34565][SQL] Collapse Window nodes with Project between them
### What changes were proposed in this pull request?

Extend the `CollapseWindow` rule to collapse `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 collapsed by the current `CollapseWindow` rule.

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

No

### How was this patch tested?

UT

Closes #31677 from tanelk/SPARK-34565_collapse_windows.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-06-21 22:10:49 +09:00
Max Gekk 37ef7bb98c [SPARK-35840][SQL] Add apply() for a single field to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to add 2 new methods that accept one field and produce either `YearMonthIntervalType` or `DayTimeIntervalType`.

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

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

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

Closes #32997 from MaxGekk/ansi-interval-types-single-field.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 14:15:33 +03:00
Angerszhuuuu 1488ea9a8c [SPARK-35820][SQL] Support Cast between different field DayTimeIntervalType
### What changes were proposed in this pull request?
 Support Cast between different field DayTimeIntervalType

### Why are the changes needed?
Make user convenient to get different field DayTimeIntervalType

### Does this PR introduce _any_ user-facing change?
User can call cast DayTimeIntervalType(DAY, SECOND) to DayTimeIntervalType(DAY, MINUTE) etc

### How was this patch tested?
Added UT

Closes #32975 from AngersZhuuuu/SPARK-35820.

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-21 12:36:38 +03:00
yi.wu 974d127c4f [SPARK-35545][FOLLOW-UP][TEST][SQL] Add a regression test for the SubqueryExpression refactor
### What changes were proposed in this pull request?

Add a test.

### Why are the changes needed?

The SubqueryExpression refactor PR https://github.com/apache/spark/pull/32687 actually fixes the bug of `SubqueryExpression.references`. So this follow-up PR adds a regression unit test for it.

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

No.

### How was this patch tested?

Added a new test.

Closes #32990 from Ngone51/spark-35545-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-21 09:54:55 +03:00
Peter Toth 682e7f2033 [SPARK-29375][SPARK-28940][SPARK-32041][SQL] Whole plan exchange and subquery reuse
### What changes were proposed in this pull request?
This PR:
1. Fixes an issue in `ReuseExchange` rule that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate traversals in `ReuseExchange` when the 2nd traversal modifies an exchange that has already been referenced (reused) in the 1st traversal.
   Consider the following query:
   ```
   WITH t AS (
     SELECT df1.id, df2.k
     FROM df1 JOIN df2 ON df1.k = df2.k
     WHERE df2.id < 2
   )
   SELECT * FROM t AS a JOIN t AS b ON a.id = b.id
   ```
   Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker):
   ```
   == Physical Plan ==
   *(7) SortMergeJoin [id#14L], [id#18L], Inner
   :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#14L, 5), true, [id=#298]
   :     +- *(2) Project [id#14L, k#17L]
   :        +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight
   :           :- *(2) Project [id#14L, k#15L]
   :           :  +- *(2) Filter isnotnull(id#14L)
   :           :     +- *(2) ColumnarToRow
   :           :        +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :           :              +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#289]
   :           :                 +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179]
   :           +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#179]
   :              +- *(1) Project [k#17L]
   :                 +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L))
   :                    +- *(1) ColumnarToRow
   :                       +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#184] <== this reuse node points to a non-existing node
   ```
   After this PR:
   ```
   == Physical Plan ==
   *(7) SortMergeJoin [id#14L], [id#18L], Inner
   :- *(3) Sort [id#14L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#14L, 5), true, [id=#231]
   :     +- *(2) Project [id#14L, k#17L]
   :        +- *(2) BroadcastHashJoin [k#15L], [k#17L], Inner, BuildRight
   :           :- *(2) Project [id#14L, k#15L]
   :           :  +- *(2) Filter isnotnull(id#14L)
   :           :     +- *(2) ColumnarToRow
   :           :        +- FileScan parquet default.df1[id#14L,k#15L] Batched: true, DataFilters: [isnotnull(id#14L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#15L), dynamicpruningexpression(k#15L IN dynamicpruning#26)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :           :              +- SubqueryBroadcast dynamicpruning#26, 0, [k#17L], [id=#103]
   :           :                 +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102]
   :           :                    +- *(1) Project [k#17L]
   :           :                       +- *(1) Filter ((isnotnull(id#16L) AND (id#16L < 2)) AND isnotnull(k#17L))
   :           :                          +- *(1) ColumnarToRow
   :           :                             +- FileScan parquet default.df2[id#16L,k#17L] Batched: true, DataFilters: [isnotnull(id#16L), (id#16L < 2), isnotnull(k#17L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :           +- ReusedExchange [k#17L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#102]
   +- *(6) Sort [id#18L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#18L, k#21L], Exchange hashpartitioning(id#14L, 5), true, [id=#231]
   ```
2. Fixes an issue with separate consecutive `ReuseExchange` and `ReuseSubquery` rules that can result a `ReusedExchange` node pointing to an invalid exchange. This can happen due to the 2 separate rules when `ReuseSubquery` rule modifies an exchange that has already been referenced (reused) in `ReuseExchange` rule.
   Consider the following query:
   ```
   WITH t AS (
     SELECT df1.id, df2.k
     FROM df1 JOIN df2 ON df1.k = df2.k
     WHERE df2.id < 2
   ),
   t2 AS (
     SELECT * FROM t
     UNION
     SELECT * FROM t
   )
   SELECT * FROM t2 AS a JOIN t2 AS b ON a.id = b.id
   ```
   Before this PR the plan of the query was (note the `<== this reuse node points to a non-existing node` marker):
   ```
   == Physical Plan ==
   *(15) SortMergeJoin [id#46L], [id#58L], Inner
   :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#46L, 5), true, [id=#979]
   :     +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :        +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#975]
   :           +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :              +- Union
   :                 :- *(2) Project [id#46L, k#49L]
   :                 :  +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                 :     :- *(2) Project [id#46L, k#47L]
   :                 :     :  +- *(2) Filter isnotnull(id#46L)
   :                 :     :     +- *(2) ColumnarToRow
   :                 :     :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                 :     :              +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                 :     :                 +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   :                 :        +- *(1) Project [k#49L]
   :                 :           +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L))
   :                 :              +- *(1) ColumnarToRow
   :                 :                 +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :                 +- *(4) Project [id#46L, k#49L]
   :                    +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                       :- *(4) Project [id#46L, k#47L]
   :                       :  +- *(4) Filter isnotnull(id#46L)
   :                       :     +- *(4) ColumnarToRow
   :                       :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                       :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#926]
   :                       +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#656]
   +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#761] <== this reuse node points to a non-existing node
   ```
   After this PR:
   ```
   == Physical Plan ==
   *(15) SortMergeJoin [id#46L], [id#58L], Inner
   :- *(7) Sort [id#46L ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(id#46L, 5), true, [id=#793]
   :     +- *(6) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :        +- Exchange hashpartitioning(id#46L, k#49L, 5), true, [id=#789]
   :           +- *(5) HashAggregate(keys=[id#46L, k#49L], functions=[])
   :              +- Union
   :                 :- *(2) Project [id#46L, k#49L]
   :                 :  +- *(2) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                 :     :- *(2) Project [id#46L, k#47L]
   :                 :     :  +- *(2) Filter isnotnull(id#46L)
   :                 :     :     +- *(2) ColumnarToRow
   :                 :     :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                 :     :              +- SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485]
   :                 :     :                 +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   :                 :     :                    +- *(1) Project [k#49L]
   :                 :     :                       +- *(1) Filter ((isnotnull(id#48L) AND (id#48L < 2)) AND isnotnull(k#49L))
   :                 :     :                          +- *(1) ColumnarToRow
   :                 :     :                             +- FileScan parquet default.df2[id#48L,k#49L] Batched: true, DataFilters: [isnotnull(id#48L), (id#48L < 2), isnotnull(k#49L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [], PushedFilters: [IsNotNull(id), LessThan(id,2), IsNotNull(k)], ReadSchema: struct<id:bigint,k:bigint>
   :                 :     +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   :                 +- *(4) Project [id#46L, k#49L]
   :                    +- *(4) BroadcastHashJoin [k#47L], [k#49L], Inner, BuildRight
   :                       :- *(4) Project [id#46L, k#47L]
   :                       :  +- *(4) Filter isnotnull(id#46L)
   :                       :     +- *(4) ColumnarToRow
   :                       :        +- FileScan parquet default.df1[id#46L,k#47L] Batched: true, DataFilters: [isnotnull(id#46L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/petertoth/git/apache/spark/sql/core/spark-warehouse/org.apache.spar..., PartitionFilters: [isnotnull(k#47L), dynamicpruningexpression(k#47L IN dynamicpruning#66)], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>
   :                       :              +- ReusedSubquery SubqueryBroadcast dynamicpruning#66, 0, [k#49L], [id=#485]
   :                       +- ReusedExchange [k#49L], BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])), [id=#484]
   +- *(14) Sort [id#58L ASC NULLS FIRST], false, 0
      +- ReusedExchange [id#58L, k#61L], Exchange hashpartitioning(id#46L, 5), true, [id=#793]
   ```
   (This example contains issue 1 as well.)

3. Improves the reuse of exchanges and subqueries by enabling reuse across the whole plan. This means that the new combined rule utilizes the reuse opportunities between parent and subqueries by traversing the whole plan. The traversal is started on the top level query only.

4. Due to the order of traversal this PR does while adding reuse nodes, the reuse nodes appear in parent queries if reuse is possible between different levels of queries (typical for DPP). This is not an issue from execution perspective, but this also means "forward references" in explain formatted output where parent queries come first. The changes I made to `ExplainUtils` are to handle these references properly.

This PR fixes the above 3 issues by unifying the separate rules into a `ReuseExchangeAndSubquery` rule that does a 1 pass, whole-plan, bottom-up traversal.

### Why are the changes needed?
Performance improvement.

### How was this patch tested?
- New UTs in `ReuseExchangeAndSubquerySuite` to cover 1. and 2.
- New UTs in `DynamicPartitionPruningSuite`, `SubquerySuite` and `ExchangeSuite` to cover 3.
- New `ReuseMapSuite` to test `ReuseMap`.
- Checked new golden files of `PlanStabilitySuite`s for invalid reuse references.
- TPCDS benchmarks.

Closes #28885 from peter-toth/SPARK-29375-SPARK-28940-whole-plan-reuse.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-21 04:53:19 +00:00
Kousuke Saruta af20474c67 [SPARK-35827][SQL] Show proper error message when update column types to year-month/day-time interval
### What changes were proposed in this pull request?

This PR fixes error message shown when changing a column type to year-month/day-time interval type is attempted.

### Why are the changes needed?

It's for consistent behavior.
Updating column types to interval types are prohibited for V2 source tables.
So, if we attempt to update the type of a column to the conventional interval type, an error message like `Error in query: Cannot update <table> field <column> to interval type;`.

But, for year-month/day-time interval types, another error message like `Error in query: Cannot update <table> field <column>:<type> cannot be cast to interval year;`.

You can reproduce with the following procedure.
```
$ bin/spark-sql
spark-sql> SET spark.sql.catalog.mycatalog=<a catalog implementation class>;
spark-sql> CREATE TABLE mycatalog.t1(c1 int) USING <V2 datasource implementation class>;
spark-sql> ALTER TABLE mycatalog.t1 ALTER COLUMN c1 TYPE interval year to month;
```

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

No.

### How was this patch tested?

Modified an existing test.

Closes #32978 from sarutak/err-msg-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-20 23:39:46 +03:00
Kousuke Saruta 4758dc78a2 [SPARK-35771][SQL][FOLLOWUP] IntervalUtils.toYearMonthIntervalString should consider the case year-month type is casted as month type
### What changes were proposed in this pull request?

This PR fixes an issue that `IntervalUtils.toYearMonthIntervalString` doesn't consider the case that year-month interval type is casted as month interval type.
If a year-month interval data is casted as month interval, the value of the year is multiplied by `12` and added to the value of month. For example, `INTERVAL '1-2' YEAR TO MONTH` will be `INTERVAL '14' MONTH` if  it's casted.
If this behavior is intended, it's stringified to be `'INTERVAL 14' MONTH` but currently, it will be `INTERVAL '2' MONTH`

### Why are the changes needed?

It's a bug if the behavior of cast is intended.

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

No, because this feature is not released yet.

### How was this patch tested?

Modified the tests added in SPARK-35771 (#32924).

Closes #32982 from sarutak/fix-toYearMonthIntervalString.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-20 10:32:21 +03:00
Angerszhuuuu 86bcd1fba0 [SPARK-35819][SQL] Support Cast between different field YearMonthIntervalType
### What changes were proposed in this pull request?
 Support Cast between different field YearMonthIntervalType

### Why are the changes needed?
Make user convenient to get different field YearMonthIntervalType

### Does this PR introduce _any_ user-facing change?
User can call cast YearMonthIntervalType(YEAR, MONTH) to YearMonthIntervalType(YEAR, YEAR) etc

### How was this patch tested?
Added UT

Closes #32974 from AngersZhuuuu/SPARK-35819.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-19 21:43:06 +03:00
Angerszhuuuu 2ebad72758 [SPARK-35726][SQL] Truncate java.time.Duration by fields of day-time interval type
### What changes were proposed in this pull request?
Support truncate java.time.Duration by fields of day-time interval type.

### Why are the changes needed?
To respect fields of the target day-time interval types.

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

### How was this patch tested?
Added UT

Closes #32950 from AngersZhuuuu/SPARK-35726.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-19 13:51:21 +03:00
Liang-Chi Hsieh 882122d6b7 [SPARK-35565][SS] Add config for ignoring metadata directory of FileStreamSink
### What changes were proposed in this pull request?

This patch proposes to add an internal config for ignoring metadata of `FileStreamSink` when reading the output path.

### Why are the changes needed?

`FileStreamSink` produces a metadata directory which logs output files per micro-batch. When we read from the output path, Spark will look at the metadata and ignore other files not in the log.

Normally it works well. But for some use-cases, we may need to ignore the metadata when reading the output path. For example, when we change the streaming query and must to run it with new checkpoint directory, we cannot use previous metadata. If we create a new metadata too, when we read the output path later in Spark, Spark only reads the files listed in the new metadata. The files written before we use new checkpoint and metadata are ignored by Spark.

Although seems we can output to different output directory every time, but it is bad idea as we will produce many directories unnecessarily.

We need a config for ignoring the metadata of `FileStreamSink` when reading the output path.

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

Added a config for ignoring metadata of FileStreamSink when reading the output.

### How was this patch tested?

Unit tests.

Closes #32702 from viirya/ignore-metadata.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-19 08:20:58 +09:00
Yuming Wang 7be8d8a164 [SPARK-35185][SQL] Improve Distinct statistics estimation
### What changes were proposed in this pull request?

This PR improves `Distinct` statistics estimation by rewrite it to `Aggregate`.

### Why are the changes needed?

1. The current implementation will lack column statistics.
2. Some rules before the `ReplaceDistinctWithAggregate` may use it. For example: https://github.com/apache/spark/pull/31113/files#diff-11264d807efa58054cca2d220aae8fba644ee0f0f2a4722c46d52828394846efR1808

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

No.

### How was this patch tested?

Unit test.

Closes #32291 from wangyum/SPARK-35185.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-06-18 21:48:44 +08:00
ulysses-you 2c4598d02e [SPARK-35608][SQL] Support AQE optimizer side transformUpWithPruning
### What changes were proposed in this pull request?

Change `AQEPropagateEmptyRelation` from `transformUp` to `transformUpWithPruning

### Why are the changes needed?

To avoid unnecessary iteration during AQE optimizer.

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

No.

### How was this patch tested?

Pass CI.

Closes #32742 from ulysses-you/aqe-transformUpWithPruning.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-18 20:31:11 +08:00
Angerszhuuuu 071566caf3 [SPARK-35769][SQL] Truncate java.time.Period by fields of year-month interval type
### What changes were proposed in this pull request?
Support truncate java.time.Period by fields of year-month interval type

### Why are the changes needed?
To follow the SQL standard and respect the field restriction of the target year-month type.

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

### How was this patch tested?
Added UT

Closes #32945 from AngersZhuuuu/SPARK-35769.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-18 11:55:57 +03:00
Kousuke Saruta 45b7f76295 [SPARK-35095][SS][TESTS] Use ANSI intervals in streaming join tests
### What changes were proposed in this pull request?

This PR extends the following tests to use day-time intervals.

* StreamingOuterJoinSuite.right outer with watermark range condition
* StreamingOuterJoinSuite.left outer with watermark range condition

### Why are the changes needed?

Currently, there are no tests to use day-time intervals.

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

No.

### How was this patch tested?

New assertions.

Closes #32953 from sarutak/stream-join-interval.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-17 22:48:18 +03:00
Gengliang Wang 05e2b76852 [SPARK-35720][SQL] Support casting of String to timestamp without time zone type
### What changes were proposed in this pull request?

Extend the Cast expression and support StringType in casting to TimestampWithoutTZType.

Closes #32898

### Why are the changes needed?

To conform the ANSI SQL standard which requires to support such casting.

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

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #32936 from gengliangwang/castStringToTswtz.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-18 02:02:10 +08:00
allisonwang-db 0d900b6cfa [SPARK-35789][SQL] Refine lateral join syntax to only allow subqueries
### What changes were proposed in this pull request?
This PR is a follow-up for SPARK-34382. It refines the lateral join syntax to only allow the LATERAL keyword to be in front of subqueries, instead of all `relationPriamry`. For example, `SELECT * FROM t1, LATERAL t2` should not be allowed.

### Why are the changes needed?
To be consistent with Postgres.

### Does this PR introduce _any_ user-facing change?
Yes. After this PR, the LATERAL keyword can only be in front of subqueries.

```scala
sql("SELECT * FROM t1, LATERAL t2")

org.apache.spark.sql.catalyst.parser.ParseException:
LATERAL can only be used with subquery(line 1, pos 26)

== SQL ==
select * from t1, lateral t2
--------------------------^^^
```

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

Closes #32937 from allisonwang-db/spark-35789-lateral-join-parser.

Authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 16:47:30 +00:00
gengjiaan ee2d8ae322 [SPARK-35378][SQL][FOLLOWUP] Move CommandResult to catalyst.plans.logical
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/32513 added the case class `CommandResult` in package `org.apache.spark.sql.expression`. It is not suitable, so this PR move `CommandResult` from `org.apache.spark.sql.expression` to `org.apache.spark.sql.catalyst.plans.logical`.

### Why are the changes needed?
Make `CommandResult` in suitable package.

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

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

Closes #32942 from beliefer/SPARK-35378-followup.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-06-17 07:47:38 -07:00
Peter Toth abf9675a75 [SPARK-35798][SQL] Fix SparkPlan.sqlContext usage
### What changes were proposed in this pull request?
There might be `SparkPlan` nodes where canonicalization on executor side can cause issues. This is a follow-up fix to conversation https://github.com/apache/spark/pull/32885/files#r651019687.

### Why are the changes needed?
To avoid potential NPEs when canonicalization happens on executors.

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

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

Closes #32947 from peter-toth/SPARK-35798-fix-sparkplan.sqlcontext-usage.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 13:49:38 +00:00
Linhong Liu b86a69f026 [SPARK-35792][SQL] View should not capture configs used in RelationConversions
### What changes were proposed in this pull request?
`RelationConversions` is actually an optimization rule while it's executed in the analysis phase.
For view, it's designed to only capture semantic configs, so we should ignore the optimization
configs that will be used in the analysis phase.

This PR also fixes the issue that view resolution will always use the default value for uncaptured config

### Why are the changes needed?
Bugfix

### Does this PR introduce _any_ user-facing change?
Yes, after this PR view resolution will respect the values set in the current session for the below configs
```
"spark.sql.hive.convertMetastoreParquet"
"spark.sql.hive.convertMetastoreOrc"
"spark.sql.hive.convertInsertingPartitionedTable"
"spark.sql.hive.convertMetastoreCtas"
```

### How was this patch tested?
By running new UT:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *HiveSQLViewSuite"
```

Closes #32941 from linhongliu-db/SPARK-35792-ignore-convert-configs.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-17 21:40:53 +08:00