Commit graph

8294 commits

Author SHA1 Message Date
Cheng Su f4c896885d [SPARK-35693][SS][TEST] Add plan check for stream-stream join unit test
### What changes were proposed in this pull request?

The changed [unit test](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala#L566) was introduce in https://github.com/apache/spark/pull/21587, to fix the planner side of thing for stream-stream join. Ideally check the query result should catch the bug, but it would be better to add plan check to make the purpose of unit test more clearly and catch future bug from planner change.

### Why are the changes needed?

Improve unit test.

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

No.

### How was this patch tested?

Changed test itself.

Closes #32836 from c21/ss-test.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 13:45:16 +00:00
Chao Sun 7d8181b62f [SPARK-35390][SQL] Handle type coercion when resolving V2 functions
### What changes were proposed in this pull request?

Handle type coercion when resolving V2 function. In particular:
- prior to evaluating function arguments, insert cast whenever the argument type doesn't match the expected input type.
- use `BoundFunction.inputTypes()` to lookup magic method for scalar function

### Why are the changes needed?

For V2 functions, the actual argument types should not necessarily match those of the input types, and Spark should handle type coercion whenever it is needed.

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

Yes. Now V2 function resolution should be able to handle type coercion properly.

### How was this patch tested?

Added a few new tests.

Closes #32764 from sunchao/SPARK-35390.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 13:22:19 +00:00
Yuming Wang ce1636948b [SPARK-35650][SQL] Enhance RepartitionByExpression to make it coalesce partitions efficiently by AQE
### What changes were proposed in this pull request?

This PR enhances `RepartitionByExpression` to make it coalesce partitions efficiently by AQE. Usually used to merge small files.
The basic logic is: Spark first tries to coalesce partitions, if it cannot be coalesced, then use the local shuffle reader to read data to avoid exchange the data over the network.

Usage:
```sql
SELECT /*+ REPARTITION */ * FROM t
```
```scala
df.repartition()
```

For example:
coalesce small output files | local shuffle reader
--- | ---
![image](https://user-images.githubusercontent.com/5399861/120772533-fc8cad00-c552-11eb-977e-5bb61b84cbe2.png)| ![image](https://user-images.githubusercontent.com/5399861/120772324-c6e7c400-c552-11eb-9daa-f6b5021fd1b9.png)

### Why are the changes needed?

Coalesce partitions efficiently.

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

No.

### How was this patch tested?

Unit test.

Closes #32781 from wangyum/SPARK-35650.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 13:16:18 +00:00
Gengliang Wang 43f6b4a810 [SPARK-35674][SQL][TESTS] Test timestamp without time zone in UDF
### What changes were proposed in this pull request?

Write tests for timestamp without time zone in UDF as input parameters and results.

### Why are the changes needed?

It follows https://github.com/apache/spark/pull/31779 to improve test coverage.

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

No
### How was this patch tested?

Unit test

Closes #32840 from gengliangwang/tswtzUDF.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-09 18:57:28 +08:00
Gengliang Wang 84c5ca33f9 [SPARK-35664][SQL] Support java.time.LocalDateTime as an external type of TimestampWithoutTZ type
### What changes were proposed in this pull request?

In the PR, I propose to extend Spark SQL API to accept `java.time.LocalDateTime` as an external type of recently added new Catalyst type - `TimestampWithoutTZ`. The Java class `java.time.LocalDateTime` has a similar semantic to ANSI SQL timestamp without timezone type, and it is the most suitable to be an external type for `TimestampWithoutTZType`. In more details:

* Added `TimestampWithoutTZConverter` which converts java.time.LocalDateTime instances to/from internal representation of the Catalyst type `TimestampWithoutTZType` (to Long type). The `TimestampWithoutTZConverter` object uses new methods of DateTimeUtils:
  * localDateTimeToMicros() converts the input date time to the total length in microseconds.
  * microsToLocalDateTime() obtains a java.time.LocalDateTime
* Support new type `TimestampWithoutTZType` in RowEncoder via the methods createDeserializerForLocalDateTime() and createSerializerForLocalDateTime().
* Extended the Literal API to construct literals from `java.time.LocalDateTime` instances.

### Why are the changes needed?

To allow users parallelization of `java.time.LocalDateTime` collections, and construct timestamp without time zone columns. Also to collect such columns back to the driver side.

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

The PR extends existing functionality. So, users can parallelize instances of the java.time.LocalDateTime class and collect them back.
```
scala> val ds = Seq(java.time.LocalDateTime.parse("1970-01-01T00:00:00")).toDS
ds: org.apache.spark.sql.Dataset[java.time.LocalDateTime] = [value: timestampwithouttz]

scala> ds.collect()
res0: Array[java.time.LocalDateTime] = Array(1970-01-01T00:00)
```
### How was this patch tested?

New unit tests

Closes #32814 from gengliangwang/LocalDateTime.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-09 14:59:46 +08:00
ulysses-you 825b620862 [SPARK-35687][SQL][TEST] PythonUDFSuite move assume into its methods
### What changes were proposed in this pull request?

Move `assume` into methods at `PythonUDFSuite`.

### Why are the changes needed?

When we run Spark test with such command:
`./build/mvn -Phadoop-2.7 -Phive -Phive-thriftserver -Pyarn -Pkubernetes clean test`

get this exception:
```
 PythonUDFSuite:
 org.apache.spark.sql.execution.python.PythonUDFSuite *** ABORTED ***
   java.lang.RuntimeException: Unable to load a Suite class that was discovered in the runpath: org.apache.spark.sql.execution.python.PythonUDFSuite
   at org.scalatest.tools.DiscoverySuite$.getSuiteInstance(DiscoverySuite.scala:81)
   at org.scalatest.tools.DiscoverySuite.$anonfun$nestedSuites$1(DiscoverySuite.scala:38)
   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
   at scala.collection.Iterator.foreach(Iterator.scala:941)
   at scala.collection.Iterator.foreach$(Iterator.scala:941)
   at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
   at scala.collection.IterableLike.foreach(IterableLike.scala:74)
   at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
   at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
```

The test env has no PYSpark module so it failed.

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

No

### How was this patch tested?

manual

Closes #32833 from ulysses-you/SPARK-35687.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-09 15:57:56 +09:00
Yuanjian Li 9f010a8eb2 [SPARK-35436][SS] RocksDBFileManager - save checkpoint to DFS
### What changes were proposed in this pull request?
The implementation for the save operation of RocksDBFileManager.

### Why are the changes needed?
Save all the files in the given local checkpoint directory as a committed version in DFS.

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

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

Closes #32582 from xuanyuanking/SPARK-35436.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-09 14:09:28 +09:00
gengjiaan 8013f985a4 [SPARK-35378][SQL] Eagerly execute commands in QueryExecution instead of caller sides
### What changes were proposed in this pull request?
Currently, Spark eagerly executes commands on the caller side of `QueryExecution`, which is a bit hacky as `QueryExecution` is not aware of it and leads to confusion.

For example, if you run `sql("show tables").collect()`, you will see two queries with identical query plans in the web UI.
![image](https://user-images.githubusercontent.com/3182036/121193729-a72d0480-c8a0-11eb-8b12-379019607ad5.png)
![image](https://user-images.githubusercontent.com/3182036/121193822-bc099800-c8a0-11eb-9d2a-34ab1329e2f7.png)
![image](https://user-images.githubusercontent.com/3182036/121193845-c0ce4c00-c8a0-11eb-96d0-ef604a4dfab0.png)

The first query is triggered at `Dataset.logicalPlan`, which eagerly executes the command.
The second query is triggered at `Dataset.collect`, which is the normal query execution.

From the web UI, it's hard to tell that these two queries are caused by eager command execution.

This PR proposes to move the eager command execution to `QueryExecution`, and turn the command plan to `CommandResult` to indicate that command has been executed already. Now `sql("show tables").collect()` still triggers two queries, but the quey plans are not identical. The second query becomes:
![image](https://user-images.githubusercontent.com/3182036/121194850-b3659180-c8a1-11eb-9abf-2980f84f089d.png)

In addition to the UI improvements, this PR also has other benefits:
1. Simplifies code as caller side no need to worry about eager command execution. `QueryExecution` takes care of it.
2. It helps https://github.com/apache/spark/pull/32442 , where there can be more plan nodes above commands, and we need to replace commands with something like local relation that produces unsafe rows.

### Why are the changes needed?
Explained above.

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

### How was this patch tested?
existing tests

Closes #32513 from beliefer/SPARK-35378.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-09 04:45:44 +00:00
Gengliang Wang 1b1a8e4eee [SPARK-30993][FOLLOWUP][SQL] Refactor LocalDateTimeUDT as YearUDT in UserDefinedTypeSuite
### What changes were proposed in this pull request?

Refactor LocalDateTimeUDT as YearUDT in UserDefinedTypeSuite

### Why are the changes needed?

As we are going to support java.time.LocalDateTime as an external type of TimestampWithoutTZ type https://github.com/apache/spark/pull/32814, registering java.time.LocalDateTime as UDT will cause test failures: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139469/testReport/
This PR is to unblock https://github.com/apache/spark/pull/32814.

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

No.

### How was this patch tested?

Unit test.

Closes #32824 from gengliangwang/UDTFollowUp.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-09 10:02:37 +08:00
Kousuke Saruta 93a9dc479c [SPARK-35602][SS] Update state schema to be able to accept long length JSON
### What changes were proposed in this pull request?

This PR fixes an issue that both key and value of state schema cannot accept long length (>65535 bytes) JSON.
To solve the problem explained below, JSON represented schema is divided into chunks whose maximum length is 65535 bytes, and each chunk is written by `DataOutputStream.writeUTF`.

As the solution changes the format of the schema, the version is also changes from `1` to `2` but old version schema is still acceptable to ensures backward compatibility.

### Why are the changes needed?

In the current implementation, writing state schema fails if the length of schema exceeds 65535 bytes and `UTFDataFormatException` is thrown.
It's due to the limitation of `DataOutputStream.writeUTF`.
`writeUTF` writes a length field first and it's 2 bytes width, meaning the maximum content length is limited to `2^16-1`=`65535` bytes.
https://docs.oracle.com/javase/8/docs/api/java/io/DataOutputStream.html#writeUTF-java.lang.String-

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

No.

### How was this patch tested?

New tests.

Closes #32788 from sarutak/fix-UTFDataFormatException.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-09 10:09:57 +09:00
Liang-Chi Hsieh 1226b9badd [SPARK-35659][SS] Avoid write null to StateStore
### What changes were proposed in this pull request?

This patch removes the usage of putting null into StateStore.

### Why are the changes needed?

According to `get` method doc in `StateStore` API, it returns non-null row if the key exists. So basically we should avoid write null to `StateStore`. You cannot distinguish if the returned null row is because the key doesn't exist, or the value is actually null. And due to the defined behavior of `get`, it is quite easy to cause NPE error if the caller doesn't expect to get a null if the caller believes the key exists.

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

No

### How was this patch tested?

Added test.

Closes #32796 from viirya/fix-ss-joinstatemanager.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-08 09:10:19 -07:00
Wenchen Fan a70e66ecfa [SPARK-35665][SQL] Resolve UnresolvedAlias in CollectMetrics
### What changes were proposed in this pull request?

It's a long-standing bug that we forgot to resolve `UnresolvedAlias` in `CollectMetrics`. It's a bit hard to trigger this bug before 3.2 as most likely people won't create `UnresolvedAlias` when calling `Dataset.observe`. However things have been changed after https://github.com/apache/spark/pull/30974

This PR proposes to handle `CollectMetrics` in the rule `ResolveAliases`.

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

updated test

Closes #32803 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 21:05:11 +09:00
Alkis Polyzotis 6f8c62047c [SPARK-35558] Optimizes for multi-quantile retrieval
### What changes were proposed in this pull request?
Optimizes the retrieval of approximate quantiles for an array of percentiles.
* Adds an overload for QuantileSummaries.query that accepts an array of percentiles and optimizes the computation to do a single pass over the sketch and avoid redundant computation.
* Modifies the ApproximatePercentiles operator to call into the new method.

All formatting changes are the result of running ./dev/scalafmt

### Why are the changes needed?
The existing implementation does repeated calls per input percentile resulting in redundant computation.

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

### How was this patch tested?
Added unit tests for the new method.

Closes #32700 from alkispoly-db/spark_35558_approx_quants_array.

Authored-by: Alkis Polyzotis <alkis.polyzotis@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-06-05 14:25:33 -05:00
Marios Meimaris b5678bee1e [SPARK-35446] Override getJDBCType in MySQLDialect to map FloatType to FLOAT
### What changes were proposed in this pull request?

Override `getJDBCType` method in `MySQLDialect` so that `FloatType` is mapped to `FLOAT` instead of `REAL`

### Why are the changes needed?

MySQL treats `REAL` as a synonym to `DOUBLE` by default (see https://dev.mysql.com/doc/refman/8.0/en/numeric-types.html). Therefore, when creating a table with a column of `REAL` type, it will be created as `DOUBLE`. However, currently, `MySQLDialect` does not provide an implementation for `getJDBCType`, and will thus ultimately fall back to `JdbcUtils.getCommonJDBCType`, which maps `FloatType` to `REAL`. This change is needed so that we can properly map the `FloatType` to `FLOAT` for MySQL.

### Does this PR introduce _any_ user-facing change?
Prior to this PR, when writing a dataframe with a `FloatType` column to a MySQL table, it will create a `DOUBLE` column. After the PR, it will create a `FLOAT` column.

### How was this patch tested?
Added a test case in `JDBCSuite` that verifies the mapping.

Closes #32605 from mariosmeim-db/SPARK-35446.

Authored-by: Marios Meimaris <marios.meimaris@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-05 12:44:16 +09:00
Kent Yao dc3317fdf9 [SPARK-21957][SQL][FOLLOWUP] Support CURRENT_USER without tailing parentheses
### What changes were proposed in this pull request?

A followup for 345d35ed1a, in this PR we support CURRENT_USER without tailing parentheses in default mode. And for ANSI mode, we can only use CURRENT_USER without tailing parentheses because it is a reserved keyword that cannot be used as a function name

### Why are the changes needed?

1. make it the same as current_date/current_timestamp
2. better ANSI compliance
### Does this PR introduce _any_ user-facing change?

no, just a followup

### How was this patch tested?

new tests

Closes #32770 from yaooqinn/SPARK-21957-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-04 13:32:56 +00:00
Ke Jia 6ce5f2491c [SPARK-35568][SQL] Add the BroadcastExchange after re-optimizing the physical plan to fix the UnsupportedOperationException when enabling both AQE and DPP
### What changes were proposed in this pull request?
This PR is to fix the `UnsupportedOperationException` described in [PR#32705](https://github.com/apache/spark/pull/32705).
When AQE and DPP are turned on at the same time, because the `BroadcastExchange` included in the DPP filter is not added through `EnsureRequirement` rule, Therefore, when AQE optimizes the DPP filter, there is no way to add `BroadcastExchange` through the `EnsureRequirement` rule in `reOptimize` method, which eventually leads to the loss of `BroadcastExchange` in the final physical plan. This PR adds `BroadcastExchange` node in the `reOptimize` method if the current plan is DPP filter.
### Why are the changes needed?
bug fix

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

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

Closes #32741 from JkSelf/fixDPP+AQEbug.

Authored-by: Ke Jia <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-04 13:29:36 +00:00
ulysses-you c7fb0e18be [SPARK-35629][SQL] Use better exception type if database doesn't exist on drop database
### What changes were proposed in this pull request?

Add database if exists check in `SeesionCatalog`

### Why are the changes needed?

Curently execute `drop database test` will throw unfriendly error msg.

```
Error in query: org.apache.hadoop.hive.metastore.api.NoSuchObjectException: test
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.metastore.api.NoSuchObjectException: test
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:112)
	at org.apache.spark.sql.hive.HiveExternalCatalog.dropDatabase(HiveExternalCatalog.scala:200)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.dropDatabase(ExternalCatalogWithListener.scala:53)
	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.dropDatabase(SessionCatalog.scala:273)
	at org.apache.spark.sql.execution.command.DropDatabaseCommand.run(ddl.scala:111)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
	at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:228)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3707)
```

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

Yes, more cleaner error msg.

### How was this patch tested?

Add test.

Closes #32768 from ulysses-you/SPARK-35629.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-04 15:52:21 +08:00
fornaix 878527d9fa [SPARK-35612][SQL] Support LZ4 compression in ORC data source
### What changes were proposed in this pull request?

This PR aims to support LZ4 compression in the ORC data source.

### Why are the changes needed?

Apache ORC supports LZ4 compression, but we cannot set LZ4 compression in the ORC data source

**BEFORE**

```scala
scala> spark.range(10).write.option("compression", "lz4").orc("/tmp/lz4")
java.lang.IllegalArgumentException: Codec [lz4] is not available. Available codecs are uncompressed, lzo, snappy, zlib, none, zstd.
```

**AFTER**

```scala
scala> spark.range(10).write.option("compression", "lz4").orc("/tmp/lz4")
```
```bash
$ orc-tools meta /tmp/lz4
Processing data file file:/tmp/lz4/part-00000-6a244eee-b092-4c79-a977-fb8a69dde2eb-c000.lz4.orc [length: 222]
Structure for file:/tmp/lz4/part-00000-6a244eee-b092-4c79-a977-fb8a69dde2eb-c000.lz4.orc
File Version: 0.12 with ORC_517
Rows: 10
Compression: LZ4
Compression size: 262144
Type: struct<id:bigint>

Stripe Statistics:
  Stripe 1:
    Column 0: count: 10 hasNull: false
    Column 1: count: 10 hasNull: false bytesOnDisk: 7 min: 0 max: 9 sum: 45

File Statistics:
  Column 0: count: 10 hasNull: false
  Column 1: count: 10 hasNull: false bytesOnDisk: 7 min: 0 max: 9 sum: 45

Stripes:
  Stripe: offset: 3 data: 7 rows: 10 tail: 35 index: 35
    Stream: column 0 section ROW_INDEX start: 3 length 11
    Stream: column 1 section ROW_INDEX start: 14 length 24
    Stream: column 1 section DATA start: 38 length 7
    Encoding column 0: DIRECT
    Encoding column 1: DIRECT_V2

File length: 222 bytes
Padding length: 0 bytes
Padding ratio: 0%

User Metadata:
  org.apache.spark.version=3.2.0
```

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

Yes.

### How was this patch tested?

Pass the newly added test case.

Closes #32751 from fornaix/spark-35612.

Authored-by: fornaix <foxnaix@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-03 14:07:26 -07:00
Liang-Chi Hsieh 0342dcb628 [SPARK-35580][SQL] Implement canonicalized method for HigherOrderFunction
### What changes were proposed in this pull request?

This patch implements `canonicalized` method for `HigherOrderFunction`. Basically it canonicalizes the name of all `NamedLambdaVariable`s and their `ExprId`. The name and `ExprId` of `NamedLambdaVariable` are unque. But to compare semantic equality between `HigherOrderFunction`, we can canonicalize them.

### Why are the changes needed?

The default `canonicalized` method does not work for `HigherOrderFunction`. It makes subexpression elimination not work for higher functions.

Manual check gen-ed code for:
```scala
val df = Seq(Seq(1, 2, 3)).toDF("a")
df.select(transform($"a", x => x + 1), transform($"a", x => x + 1)).collect()
```

The code for `transform(input[0, array<int>, true], lambdafunction((lambda x_20#19041 + 1), lambda x_20#19041, false)),transform(input[0, array<int>, true], lambdafunction((lambda x_21#19042 + 1), lambda x_21#19042, false))`, generated by `GenerateUnsafeProjection`.

Before:

```java
/* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
/* 028 */   public UnsafeRow apply(InternalRow i) {
...
/* 034 */     Object obj_0 = ((Expression) references[0]).eval(i);
...
/* 062 */     Object obj_1 = ((Expression) references[1]).eval(i);
...
/* 093 */ }
```

After:
```java
/* 005 */ class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
/* 031 */   public UnsafeRow apply(InternalRow i) {
...
/* 033 */     subExpr_0(i);
...
/* 086 */   private void subExpr_0(InternalRow i) {
/* 087 */     Object obj_0 = ((Expression) references[0]).eval(i);
/* 088 */     boolean isNull_0 = obj_0 == null;
/* 089 */     ArrayData value_0 = null;
/* 090 */     if (!isNull_0) {
/* 091 */       value_0 = (ArrayData) obj_0;
/* 092 */     }
/* 093 */     subExprIsNull_0 = isNull_0;
/* 094 */     mutableStateArray_0[0] = value_0;
/* 095 */   }
/* 096 */
/* 097 */ }
```

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

No

### How was this patch tested?

Unit test and manual check gen-ed code.

Closes #32735 from viirya/higher-func-canonicalize.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-03 09:16:47 -07:00
Kent Yao 345d35ed1a [SPARK-21957][SQL] Support current_user function
### What changes were proposed in this pull request?

Currently, we do not have a suitable definition of the `user` concept in Spark. We only have a `sparkUser` app widely but do not support identify or retrieve the user information from a session in STS or a runtime query execution.

`current_user()` is very popular and supported by plenty of other modern or old school databases, and also ANSI compliant.

This PR add `current_user()`  as a SQL function. And, they are the same.  In this PR, we add these functions w/o ambiguity.
1. For a normal single-threaded Spark application, clearly the `sparkUser` is always equivalent to `current_user()` .
2. For a multi-threaded Spark application, e.g. Spark thrift server, we use a `ThreadLocal` variable to store the client-side user(after authenticated) before running the query and retrieve it in the parser.

### Why are the changes needed?

`current_user()` is very popular and supported by plenty of other modern or old school databases, and also ANSI compliant.

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

yes, added  `current_user()`  as a SQL function
### How was this patch tested?

new tests in thrift server and sql/catalyst

Closes #32718 from yaooqinn/SPARK-21957.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-02 13:04:40 +00:00
ulysses-you daf9d198dc [SPARK-35585][SQL] Support propagate empty relation through project/filter
### What changes were proposed in this pull request?

Add rule `ConvertToLocalRelation` into AQE Optimizer.

### Why are the changes needed?

Support propagate empty local relation through project and filter like such SQL case:
```
Aggregate
  Project
    Join
      ShuffleStage
      ShuffleStage
```

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

No.

### How was this patch tested?

Add test.

Closes #32724 from ulysses-you/SPARK-35585.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-02 07:49:56 +00:00
Cheng Su 54e9999d39 [SPARK-35604][SQL] Fix condition check for FULL OUTER sort merge join
### What changes were proposed in this pull request?

The condition check for FULL OUTER sort merge join (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala#L1368 ) has unnecessary trip when `leftIndex == leftMatches.size` or `rightIndex == rightMatches.size`. Though this does not affect correctness (`scanNextInBuffered()` returns false anyway). But we can avoid it in the first place.

### Why are the changes needed?

Better readability for developers and avoid unnecessary execution.

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

No.

### How was this patch tested?

Existing unit tests, such as `OuterJoinSuite.scala`.

Closes #32736 from c21/join-bug.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-06-02 14:01:34 +08:00
itholic 48252bac95 [SPARK-35583][DOCS] Move JDBC data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

- "JDBC To Other Databases" page
<img width="803" alt="Screen Shot 2021-06-02 at 11 34 14 AM" src="https://user-images.githubusercontent.com/44108233/120415520-a115c000-c396-11eb-9663-9e666e08ed2b.png">

- Python
![Screen Shot 2021-06-01 at 2 57 40 PM](https://user-images.githubusercontent.com/44108233/120273628-ba146780-c2e9-11eb-96a8-11bd25415197.png)

- Scala
![Screen Shot 2021-06-01 at 2 57 03 PM](https://user-images.githubusercontent.com/44108233/120273567-a2d57a00-c2e9-11eb-9788-ea58028ca0a6.png)

- Java
![Screen Shot 2021-06-01 at 2 58 27 PM](https://user-images.githubusercontent.com/44108233/120273722-d912f980-c2e9-11eb-83b3-e09992d8c582.png)

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32723 from itholic/SPARK-35583.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-02 14:21:16 +09:00
Yingyi Bu 3f6322f9aa [SPARK-35077][SQL] Migrate to transformWithPruning for leftover optimizer rules
### What changes were proposed in this pull request?

Migrate to transformWithPruning for the following queries:
- SimplifyExtractValueOps
- NormalizeFloatingNumbers
- PushProjectionThroughUnion
- PushDownPredicates
- ExtractPythonUDFFromAggregate
- ExtractPythonUDFFromJoinCondition
- ExtractGroupingPythonUDFFromAggregate
- ExtractPythonUDFs
- CleanupDynamicPruningFilters

</google-sheets-html-origin>

### Why are the changes needed?

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

### How was this patch tested?

Existing tests.
Performance diff:
<google-sheets-html-origin><style type="text/css"></style>
&nbsp; | Baseline | Experiment | Experiment/Baseline
-- | -- | -- | --
SimplifyExtractValueOps | 99367049 | 3679579 | 0.04
NormalizeFloatingNumbers | 24717928 | 20451094 | 0.83
PushProjectionThroughUnion | 14130245 | 7913551 | 0.56
PushDownPredicates | 276333542 | 261246842 | 0.95
ExtractPythonUDFFromAggregate | 6459451 | 2683556 | 0.42
ExtractPythonUDFFromJoinCondition | 5695404 | 2504573 | 0.44
ExtractGroupingPythonUDFFromAggregate | 5546701 | 1858755 | 0.34
ExtractPythonUDFs | 58726458 | 1598518 | 0.03
CleanupDynamicPruningFilters | 26606652 | 15417936 | 0.58
OptimizeSubqueries | 3072287940 | 2876462708 | 0.94

</google-sheets-html-origin>

Closes #32721 from sigmod/pushdown.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-02 11:46:33 +08:00
Liang-Chi Hsieh dbf0b50757 [SPARK-35560][SQL] Remove redundant subexpression evaluation in nested subexpressions
### What changes were proposed in this pull request?

This patch proposes to improve subexpression evaluation under whole-stage codegen for the cases of nested subexpressions.

### Why are the changes needed?

In the cases of nested subexpressions, whole-stage codegen's subexpression elimination will do redundant subexpression evaluation. We should reduce it. For example, if we have two sub-exprs:

1. `simpleUDF($"id")`
2. `functions.length(simpleUDF($"id"))`

We should only evaluate `simpleUDF($"id")` once, i.e.

```java
subExpr1 = simpleUDF($"id");
subExpr2 = functions.length(subExpr1);
```

Snippets of generated codes:

Before:
```java
/* 040 */   private int project_subExpr_1(long project_expr_0_0) {
/* 041 */     boolean project_isNull_6 = false;
/* 042 */     UTF8String project_value_6 = null;
/* 043 */     if (!false) {
/* 044 */       project_value_6 = UTF8String.fromString(String.valueOf(project_expr_0_0));
/* 045 */     }
/* 046 */
/* 047 */     Object project_arg_1 = null;
/* 048 */     if (project_isNull_6) {
/* 049 */       project_arg_1 = ((scala.Function1[]) references[3] /* converters */)[0].apply(null);
/* 050 */     } else {
/* 051 */       project_arg_1 = ((scala.Function1[]) references[3] /* converters */)[0].apply(project_value_6);                                                              /* 052 */     }
/* 053 */
/* 054 */     UTF8String project_result_1 = null;                                                                                                                            /* 055 */     try {                                                                                                                                                          /* 056 */       project_result_1 = (UTF8String)((scala.Function1[]) references[3] /* converters */)[1].apply(((scala.Function1) references[4] /* udf */).apply(project_arg_1)
);
/* 057 */     } catch (Throwable e) {
/* 058 */       throw QueryExecutionErrors.failedExecuteUserDefinedFunctionError(
/* 059 */         "DataFrameSuite$$Lambda$6418/1507986601", "string", "string", e);
/* 060 */     }
/* 061 */
/* 062 */     boolean project_isNull_5 = project_result_1 == null;
/* 063 */     UTF8String project_value_5 = null;
/* 064 */     if (!project_isNull_5) {
/* 065 */       project_value_5 = project_result_1;
/* 066 */     }
/* 067 */     boolean project_isNull_4 = project_isNull_5;
/* 068 */     int project_value_4 = -1;
/* 069 */
/* 070 */     if (!project_isNull_5) {
/* 071 */       project_value_4 = (project_value_5).numChars();
/* 072 */     }
/* 073 */     project_subExprIsNull_1 = project_isNull_4;
/* 074 */     return project_value_4;
/* 075 */   }
...
/* 149 */   private UTF8String project_subExpr_0(long project_expr_0_0) {
/* 150 */     boolean project_isNull_2 = false;
/* 151 */     UTF8String project_value_2 = null;
/* 152 */     if (!false) {
/* 153 */       project_value_2 = UTF8String.fromString(String.valueOf(project_expr_0_0));
/* 154 */     }
/* 155 */
/* 156 */     Object project_arg_0 = null;
/* 157 */     if (project_isNull_2) {
/* 158 */       project_arg_0 = ((scala.Function1[]) references[1] /* converters */)[0].apply(null);
/* 159 */     } else {
/* 160 */       project_arg_0 = ((scala.Function1[]) references[1] /* converters */)[0].apply(project_value_2);
/* 161 */     }
/* 162 */
/* 163 */     UTF8String project_result_0 = null;
/* 164 */     try {
/* 165 */       project_result_0 = (UTF8String)((scala.Function1[]) references[1] /* converters */)[1].apply(((scala.Function1) references[2] /* udf */).apply(project_arg_0)
);
/* 166 */     } catch (Throwable e) {
/* 167 */       throw QueryExecutionErrors.failedExecuteUserDefinedFunctionError(
/* 168 */         "DataFrameSuite$$Lambda$6418/1507986601", "string", "string", e);
/* 169 */     }
/* 170 */
/* 171 */     boolean project_isNull_1 = project_result_0 == null;                                                                                                           /* 172 */     UTF8String project_value_1 = null;                                                                                                                             /* 173 */     if (!project_isNull_1) {                                                                                                                                       /* 174 */       project_value_1 = project_result_0;
/* 175 */     }
/* 176 */     project_subExprIsNull_0 = project_isNull_1;
/* 177 */     return project_value_1;
/* 178 */   }
```

After:
```java
/* 041 */   private void project_subExpr_1(long project_expr_0_0) {
/* 042 */     boolean project_isNull_8 = project_subExprIsNull_0;
/* 043 */     int project_value_8 = -1;
/* 044 */
/* 045 */     if (!project_subExprIsNull_0) {
/* 046 */       project_value_8 = (project_mutableStateArray_0[0]).numChars();
/* 047 */     }
/* 048 */     project_subExprIsNull_1 = project_isNull_8;
/* 049 */     project_subExprValue_0 = project_value_8;
/* 050 */   }
/* 056 */
...
/* 123 */
/* 124 */   private void project_subExpr_0(long project_expr_0_0) {
/* 125 */     boolean project_isNull_6 = false;
/* 126 */     UTF8String project_value_6 = null;
/* 127 */     if (!false) {
/* 128 */       project_value_6 = UTF8String.fromString(String.valueOf(project_expr_0_0));
/* 129 */     }
/* 130 */
/* 131 */     Object project_arg_1 = null;
/* 132 */     if (project_isNull_6) {
/* 133 */       project_arg_1 = ((scala.Function1[]) references[3] /* converters */)[0].apply(null);
/* 134 */     } else {
/* 135 */       project_arg_1 = ((scala.Function1[]) references[3] /* converters */)[0].apply(project_value_6);
/* 136 */     }
/* 137 */
/* 138 */     UTF8String project_result_1 = null;
/* 139 */     try {
/* 140 */       project_result_1 = (UTF8String)((scala.Function1[]) references[3] /* converters */)[1].apply(((scala.Function1) references[4] /* udf */).apply(project_arg_1)
);
/* 141 */     } catch (Throwable e) {
/* 142 */       throw QueryExecutionErrors.failedExecuteUserDefinedFunctionError(
/* 143 */         "DataFrameSuite$$Lambda$6430/2004847941", "string", "string", e);
/* 144 */     }
/* 145 */
/* 146 */     boolean project_isNull_5 = project_result_1 == null;
/* 147 */     UTF8String project_value_5 = null;
/* 148 */     if (!project_isNull_5) {
/* 149 */       project_value_5 = project_result_1;
/* 150 */     }
/* 151 */     project_subExprIsNull_0 = project_isNull_5;
/* 152 */     project_mutableStateArray_0[0] = project_value_5;
/* 153 */   }
```

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

No

### How was this patch tested?

Unit test.

Closes #32699 from viirya/improve-subexpr.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-06-01 19:13:12 -07:00
Gengliang Wang 9d0d4edb43 [SPARK-35595][TESTS] Support multiple loggers in testing method withLogAppender
### What changes were proposed in this pull request?

A test case of AdaptiveQueryExecSuite becomes flaky since there are too many debug logs in RootLogger:
https://github.com/Yikun/spark/runs/2715222392?check_suite_focus=true
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139125/testReport/

To fix it,  I suggest supporting multiple loggers in the testing method withLogAppender. So that the LogAppender gets clean target log outputs.

### Why are the changes needed?

Fix a flaky test case.
Also, reduce unnecessary memory cost in tests.

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

No
### How was this patch tested?

Unit test

Closes #32725 from gengliangwang/fixFlakyLogAppender.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-02 10:05:29 +08:00
Gengliang Wang 6a277bb7c6 [SPARK-35600][TESTS] Move Set command related test cases to SetCommandSuite
### What changes were proposed in this pull request?

Move `Set` command related test cases from `SQLQuerySuite` to a new test suite `SetCommandSuite`. There are 7 test cases in total.

### Why are the changes needed?

Code refactoring. `SQLQuerySuite` is becoming big.

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

No

### How was this patch tested?

Unit tests

Closes #32732 from gengliangwang/setsuite.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-02 10:36:21 +09:00
Max Gekk a59063d544 [SPARK-35581][SQL] Support special datetime values in typed literals only
### What changes were proposed in this pull request?
In the PR, I propose to support special datetime values introduced by #25708 and by #25716 only in typed literals, and don't recognize them in parsing strings to dates/timestamps. The following string values are supported only in typed timestamp literals:
- `epoch [zoneId]` - `1970-01-01 00:00:00+00 (Unix system time zero)`
- `today [zoneId]` - midnight today.
- `yesterday [zoneId]` - midnight yesterday
- `tomorrow [zoneId]` - midnight tomorrow
- `now` - current query start time.

For example:
```sql
spark-sql> SELECT timestamp 'tomorrow';
2019-09-07 00:00:00
```

Similarly, the following special date values are supported only in typed date literals:
- `epoch [zoneId]` - `1970-01-01`
- `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`.
- `yesterday [zoneId]` - the current date -1
- `tomorrow [zoneId]` - the current date + 1
- `now` - the date of running the current query. It has the same notion as `today`.

For example:
```sql
spark-sql> SELECT date 'tomorrow' - date 'yesterday';
2
```

### Why are the changes needed?
In the current implementation, Spark supports the special date/timestamp value in any input strings casted to dates/timestamps that leads to the following problems:
- If executors have different system time, the result is inconsistent, and random. Column values depend on where the conversions were performed.
- The special values play the role of distributed non-deterministic functions though users might think of the values as constants.

### Does this PR introduce _any_ user-facing change?
Yes but the probability should be small.

### How was this patch tested?
By running existing test suites:
```
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
```

Closes #32714 from MaxGekk/remove-datetime-special-values.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-06-01 15:29:05 +03:00
itholic 73d4f67145 [SPARK-35433][DOCS] Move CSV data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

- "CSV Files" page
<img width="970" alt="Screen Shot 2021-05-27 at 12 35 36 PM" src="https://user-images.githubusercontent.com/44108233/119762269-586a8c80-bee8-11eb-8443-ae5b3c7a685c.png">

- Python
<img width="785" alt="Screen Shot 2021-05-25 at 4 12 10 PM" src="https://user-images.githubusercontent.com/44108233/119455390-83cc6a80-bd74-11eb-9156-65785ae27db0.png">

- Scala
<img width="718" alt="Screen Shot 2021-05-25 at 4 12 39 PM" src="https://user-images.githubusercontent.com/44108233/119455414-89c24b80-bd74-11eb-9775-aeda549d081e.png">

- Java
<img width="667" alt="Screen Shot 2021-05-25 at 4 13 09 PM" src="https://user-images.githubusercontent.com/44108233/119455422-8d55d280-bd74-11eb-97e8-86c1eabeadc2.png">

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32658 from itholic/SPARK-35433.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-01 10:58:49 +09:00
Gengliang Wang 8e11f5f007 [SPARK-35576][SQL] Redact the sensitive info in the result of Set command
### What changes were proposed in this pull request?

Currently, the results of following SQL queries are not redacted:
```
SET [KEY];
SET;
```
For example:

```
scala> spark.sql("set javax.jdo.option.ConnectionPassword=123456").show()
+--------------------+------+
|                 key| value|
+--------------------+------+
|javax.jdo.option....|123456|
+--------------------+------+

scala> spark.sql("set javax.jdo.option.ConnectionPassword").show()
+--------------------+------+
|                 key| value|
+--------------------+------+
|javax.jdo.option....|123456|
+--------------------+------+

scala> spark.sql("set").show()
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|javax.jdo.option....|              123456|

```

We should hide the sensitive information and redact the query output.

### Why are the changes needed?

Security.

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

Yes, the sensitive information in the output of Set commands are redacted

### How was this patch tested?

Unit test

Closes #32712 from gengliangwang/redactSet.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-31 14:50:18 -07:00
shahid cd2ef9cb43 [SPARK-35567][SQL] Fix: Explain cost is not showing statistics for all the nodes
### What changes were proposed in this pull request?
Explain cost command in spark currently doesn't show statistics for all the nodes. It misses some nodes in almost all the TPCDS queries.
In this PR, we are collecting all the plan nodes including the subqueries and computing  the statistics for each node, if it doesn't exists in stats cache,

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

Query15, Aggregate node doesn't have statistics

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

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

### How was this patch tested?
Manual testing

Closes #32704 from shahidki31/shahid/fixshowstats.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-06-01 00:55:29 +08:00
Liang-Chi Hsieh 73ba4492b1 [SPARK-35566][SS] Fix StateStoreRestoreExec output rows
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing tests.

Closes #32703 from viirya/fix-outputrows.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Migrated `transformAllExpressions` call sites to use `transformAllExpressionsWithPruning`

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #32643 from sigmod/all_expressions.

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

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

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

### Why are the changes needed?

speed up the test

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

no

### How was this patch tested?

N/A

Closes #32695 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-28 12:39:34 -07:00
ulysses-you 3b94aad5e7 [SPARK-35552][SQL] Make query stage materialized more readable
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Pass CI.

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

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

Various small code simplification/cleanup for OptimizeSkewedJoin

### Why are the changes needed?

code refactor

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

no

### How was this patch tested?

existing tests

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

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

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

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

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

Closes #32272 from xuanyuanking/SPARK-35172.

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

However, seems Spark has different behavior in two cases:

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

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

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

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

### How was this patch tested?
Add test

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Unit test

Closes #32678 from gengliangwang/LocalScan.

Lead-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-27 19:31:56 +09:00
Cheng Su 5cc17ba0c7 [SPARK-35351][SQL][FOLLOWUP] Avoid using loaded variable for LEFT ANTI SMJ code-gen
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Existing unite tests in `ExistenceJoinSuite`.

Closes #32681 from c21/join-followup.

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

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

### Why are the changes needed?

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

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

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

Yes, add a new config.

### How was this patch tested?

Add test.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Changed unit test in `SQLMetricsSuite.scala`.

Closes #32671 from c21/agg-metrics.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 11:28:12 +00:00
itholic 79a6b0cc8a [SPARK-35509][DOCS] Move text data source options from Python and Scala into a single page
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

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

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

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

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32660 from itholic/SPARK-35509.

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

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

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

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

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

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

Closes #32636 from vinodkc/br_build_upgrade_json4s.

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

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

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

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

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

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-26 04:40:53 +00:00
Wenchen Fan 859a53424a [SPARK-35447][SQL] Optimize skew join before coalescing shuffle partitions
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

No

### How was this patch tested?

new UT and existing tests

Closes #32594 from cloud-fan/shuffle.

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

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

### Why are the changes needed?

Make `EliminateUnnecessaryJoin` available with more case.

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

No.

### How was this patch tested?

Add test.

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

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-25 08:59:59 +00:00
Kousuke Saruta d4fb98354a [SPARK-35287][SQL] Allow RemoveRedundantProjects to preserve ProjectExec which generates UnsafeRow for DataSourceV2ScanRelation
### What changes were proposed in this pull request?

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

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

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

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

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

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

        val df = spark.read.format("iceberg").load(path)
        val dfX = df.as("x")
        val dfY = df.as("y")
        val join = dfX.filter(dfX("id") > 0).join(dfY, "id")
        join.explain("extended")
        assert(join.count() == 2)
      }
    }
  }
}
```
```
[info] - RemoveRedundantProjects removes non-redundant projects *** FAILED ***
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 4) (xeroxms100.northamerica.corp.microsoft.com executor driver): java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
[info]  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:226)
[info]  at org.apache.spark.sql.execution.SortExec.$anonfun$doExecute$1(SortExec.scala:119)
```

### Why are the changes needed?

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

No.

### How was this patch tested?

New test.

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Existing tests.

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

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

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Updated existing test with new case.

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Added tests.

Closes #32559 from viirya/SPARK-35410.

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

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

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

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

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

**Before:**

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

**After:**

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

### Why are the changes needed?

To make percentile_approx work with pivot as expected

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

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

### How was this patch tested?

Manually tested and unit test was added.

Closes #32619 from HyukjinKwon/SPARK-35480.

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

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

### Why are the changes needed?

fix mistake

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

no

### How was this patch tested?

existing tests

Closes #32618 from cloud-fan/follow1.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-22 10:33:57 +08:00
Dongjoon Hyun cc05daa884 [SPARK-34558][SQL][TESTS][FOLLOWUP] Fix a test to use a unknown filesystem
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Pass the CIs.

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

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

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

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

a new test case

Closes #32622 from cloud-fan/follow2.

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

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

### Why are the changes needed?

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

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

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

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

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

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

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

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32161 from itholic/SPARK-34491.

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

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

### Why are the changes needed?

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

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

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

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

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

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

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

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32204 from itholic/SPARK-35081.

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

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

### Why are the changes needed?

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

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

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

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

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

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

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

### How was this patch tested?

Manually build docs and confirm the page.

Closes #32546 from itholic/SPARK-35395.

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

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

### Why are the changes needed?

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

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

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

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

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

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

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

No.

### How was this patch tested?

Added unit tests.

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

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

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

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

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

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

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

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

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

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

Closes #32478 from beliefer/SPARK-35063.

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

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

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

### Why are the changes needed?

To improve explain strings.

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

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

### How was this patch tested?

Added tests in `SQLQueryTestSuite`.

Closes #32615 from maropu/ExplainPartitionFilters.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-21 05:45:45 +00:00
Kent Yao 2e9936db93 [SPARK-35456][CORE] Print the invalid value in config validation error message
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

yes, modified tests

Closes #32600 from yaooqinn/SPARK-35456.

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

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

### Why are the changes needed?

Hive already handled this scenario: HIVE-11319

Steps to reproduce:

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

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

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

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

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

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

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

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

Closes #32411 from vinodkc/br_fixCTAS_nonempty_dir.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Existing tests.

Closes #32585 from c21/minor-doc.

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

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

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

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

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

Closes #32195 from andygrove/SPARK-35093.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-05-19 07:45:26 -05:00
Kousuke Saruta 9283bebbbd [SPARK-35418][SQL] Add sentences function to functions.{scala,py}
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

New test.

Closes #32566 from sarutak/sentences-function.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-19 20:07:28 +09:00
Yuzhou Sun a72d05c7e6 [SPARK-35106][CORE][SQL] Avoid failing rename caused by destination directory not exist
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

No

### How was this patch tested?

Modified DebugFilesystem#rename, and added new unit tests.

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

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

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

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

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

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

Closes #32530 from YuzhouSun/SPARK-35106.

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

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

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

### Why are the changes needed?

For better performance.

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

No.

### How was this patch tested?

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

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

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 06:19:01 +00:00
Takeshi Yamamuro 746d80d87a [SPARK-35422][SQL] Fix plan-printing issues to pass the TPCDS plan stability tests in Scala v2.13
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

To pass the tests in Scala v2.13.

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

Yes, this fix changes query explain strings.

### How was this patch tested?

Manually checked.

Closes #32577 from maropu/FixTPCDSTestIssueInScala213.

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Added new tests.

Closes #32553 from sunchao/SPARK-35389.

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

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

Example query:

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

Example generated code:

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

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

### Why are the changes needed?

Improve the query CPU performance.

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

No.

### How was this patch tested?

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

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

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

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

### Why are the changes needed?

To pass the test in Scala v2.13.

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

No, dev-only.

### How was this patch tested?

Manually checked.

Closes #32578 from maropu/FixSQLTestIssueInScala213.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-17 22:51:32 -07:00
Chao Sun a60c36458d [SPARK-34981][SQL][TESTS][FOLLOWUP] Fix test failure under Scala 2.13
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

No

### How was this patch tested?

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

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

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

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

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

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

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

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

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

Closes #32501 from fhygh/master.

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

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

### Why are the changes needed?

better error message

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

no

### How was this patch tested?

updated tests

Closes #32503 from cloud-fan/try.

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

New test suite added.

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

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

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

### Why are the changes needed?

To recover compilation.

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

No.

### How was this patch tested?

Closes #32568 from wangyum/HOT-FIX.

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

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

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

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

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

### Why are the changes needed?

Improve query performance.

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

No.

### How was this patch tested?

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Closes #29642 from wangyum/SPARK-32792.

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

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

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added a unit test.

Closes #32560 from maropu/FixIntervalIssue.

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

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

### Why are the changes needed?

Fix the potential reliability issue.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32548 from c21/smj-followup.

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

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

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

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

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

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

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-16 09:03:26 -05:00
QuangHuyViettel 9789ee84e4 [SPARK-32484][SQL] Fix log info BroadcastExchangeExec.scala
### What changes were proposed in this pull request?
Fix log info in BroadcastExchangeExec.scala

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

### How was this patch tested?
no

Closes #32544 from LittleCuteBug/SPARK-32484.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing tests.

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

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Pass existing tests.

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

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

Refine comment in `CacheManager`.

### Why are the changes needed?

Avoid misleading developer.

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

No.

### How was this patch tested?

Not needed.

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

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-05-14 17:10:21 +08:00
Takeshi Yamamuro 8fa739fb9d [SPARK-35329][SQL] Split generated switch code into pieces in ExpandExec
### What changes were proposed in this pull request?

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

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

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

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

### Why are the changes needed?

For better generated code.

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

No.

### How was this patch tested?

GA passed.

Closes #32457 from maropu/splitSwitchCode.

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

### Why are the changes needed?
Code refactor

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

### How was this patch tested?
exsting UT

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

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

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

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

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

Closes #32464 from beliefer/SPARK-35062.

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

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

### Why are the changes needed?

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

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

Yes, a new config.

### How was this patch tested?

Add test.

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

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

Add New SQL functions:
* TRY_ADD
* TRY_DIVIDE

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

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

### Why are the changes needed?

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

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

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

Yes, new SQL functions: TRY_ADD/TRY_DIVIDE

### How was this patch tested?

Unit test

Closes #32292 from gengliangwang/try_add.

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

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

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

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

Closes #31756 from JkSelf/supportDPP2.

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

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

new tests

Closes #32192 from cloud-fan/merge.

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

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

Example query:

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

Example of generated code for the query:

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

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

### Why are the changes needed?

Improve query CPU performance. Test with one query:

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

Seeing 30% of run-time improvement:

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

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

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

No.

### How was this patch tested?

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

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

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

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

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

### Why are the changes needed?

better use experience

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

no, dev change

### How was this patch tested?

new tests

Closes #32515 from yaooqinn/SPARK-35380.

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

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

### Why are the changes needed?

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

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

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

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

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

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

No

### How was this patch tested?

Existing tests.

Closes #32527 from sunchao/SPARK-35384.

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

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

### Why are the changes needed?

To improve test performance.

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

No, dev only.

### How was this patch tested?

Existing tests.

Closes #32520 from maropu/SkipDupQueries.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

N/A

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

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:41:12 -07:00
Cheng Su 7bcadedbd2 [SPARK-35349][SQL] Add code-gen for left/right outer sort merge join
### What changes were proposed in this pull request?

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

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

Example query:

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

Example generated code:

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

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

### Why are the changes needed?

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

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

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

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

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

No.

### How was this patch tested?

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

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

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-12 14:10:15 +00:00
PengLei 82c520a3e2 [SPARK-35243][SQL] Support columnar execution on ANSI interval types
### What changes were proposed in this pull request?
Columnar execution support for ANSI interval types include YearMonthIntervalType and DayTimeIntervalType

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

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

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

No

### How was this patch tested?

Existing tests

Closes #32507 from sunchao/SPARK-35361.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-12 10:16:35 +09:00
Cheng Su c4ca23207b [SPARK-35363][SQL] Refactor sort merge join code-gen be agnostic to join type
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

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

Closes #32495 from c21/smj-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-11 11:21:59 +09:00
Angerszhuuuu 7182f8cece [SPARK-35360][SQL] RepairTableCommand respects spark.sql.addPartitionInBatch.size too
### What changes were proposed in this pull request?
RepairTableCommand respects `spark.sql.addPartitionInBatch.size` too

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

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

### How was this patch tested?
Not need

Closes #32489 from AngersZhuuuu/SPARK-35360.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

N/A

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

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-10 07:20:05 +00:00
Cheng Su 38eb5a6936 [SPARK-35354][SQL] Replace BaseJoinExec with ShuffledJoin in CoalesceBucketsInJoin
### What changes were proposed in this pull request?

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

### Why are the changes needed?

Code cleanup.

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

No.

### How was this patch tested?

Existing unit test in `CoalesceBucketsInJoinSuite`.

Closes #32480 from c21/minor-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-10 10:04:49 +09:00
Takeshi Yamamuro 06c40091a6 [SPARK-35327][SQL][TESTS] Filters out the TPC-DS queries that can cause flaky test results
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

For stable testing.

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

No, dev-only.

### How was this patch tested?

GA passed.

Closes #32454 from maropu/CleanUpTpcdsQueries.

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

This PR makes the below case work well.

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

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

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

bugfix

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

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

### How was this patch tested?

new tests

Closes #32465 from yaooqinn/SPARK-35331.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-08 05:00:51 -07:00
Chao Sun f47e0f8379 [SPARK-35261][SQL] Support static magic method for stateless Java ScalarFunction
### What changes were proposed in this pull request?

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

Added new UT.

Closes #32407 from sunchao/SPARK-35261.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-07 20:34:51 -07:00
Wenchen Fan e83910f1f8 [SPARK-26164][SQL][FOLLOWUP] WriteTaskStatsTracker should know which file the row is written to
### What changes were proposed in this pull request?

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

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

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

### Why are the changes needed?

To not break some custom `WriteTaskStatsTracker` implementations.

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

no

### How was this patch tested?

N/A

Closes #32459 from cloud-fan/minor.

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

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

### Why are the changes needed?

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

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

No.

### How was this patch tested?

Added new tsts

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

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

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

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

### Why are the changes needed?

Make `EXPLAIN CODEGEN` work same as before.

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

No (when comparing with latest Spark release 3.1.1).

### How was this patch tested?

Added unit test in `ExplainSuite.scala`.

Closes #32430 from c21/explain-aqe.

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

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

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

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

Closes #31638 from xuanyuanking/SPARK-34526.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing unit test.

Closes #32330 from viirya/metric-update.

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

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

### Why are the changes needed?

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

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

No

### How was this patch tested?

Existing unit tests.

Closes #32361 from viirya/checkpoint-manager.

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

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

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

### How was this patch tested?
existing UT

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

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

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

### Why are the changes needed?

For better testing.

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

No.

### How was this patch tested?

GA passed.

Closes #32420 from maropu/UseTpcdsKit.

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

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

### Why are the changes needed?

This will provide a test coverage for this feature.

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

No.

### How was this patch tested?

Pass the CIs with the newly added test case.

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

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-05 22:29:54 -07:00
Yijia Cui bbdbe0f734 [SPARK-34854][SQL][SS] Expose source metrics via progress report and add Kafka use-case to report delay
### What changes were proposed in this pull request?
This pull request proposes a new API for streaming sources to signal that they can report metrics, and adds a use case to support Kafka micro batch stream to report the stats of # of offsets for the current offset falling behind the latest.

A public interface is added.

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

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

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

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

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

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

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

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

### Why are the changes needed?

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

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

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

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

No

### How was this patch tested?

Added the new test in `DataFrameFunctionsSuite`.

Closes #32424 from maropu/pr31887.

Lead-authored-by: dsolow <dsolow@sayari.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: dmsolow <dsolow@sayarianalytics.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-05 12:46:13 +09:00
HyukjinKwon 8aaa9e890a [SPARK-35250][SQL][DOCS] Fix duplicated STOP_AT_DELIMITER to SKIP_VALUE at CSV's unescapedQuoteHandling option documentation
### What changes were proposed in this pull request?

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

### Why are the changes needed?

To correctly document.

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

Yes, it fixes the user-facing documentation.

### How was this patch tested?

I checked them via running linters.

Closes #32423 from HyukjinKwon/SPARK-35250.

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

Fixing some typos in the documenting comments.

### Why are the changes needed?

To make reading the docs more pleasant.

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

Yes, since the user sees the docs.

### How was this patch tested?

It was not tested, because no code was changed.

Closes #32400 from Dobiasd/patch-1.

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

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

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

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

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

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

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

The code authors of these classes are:
juliuszsompolski
npoggi
wangyum

### Why are the changes needed?

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

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

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

dev only.

### How was this patch tested?

Manually checked and GA passed.

Closes #32243 from maropu/tpcdsDatagen.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-03 12:04:42 +09:00
Peter Toth cfc0495f9c [SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function
### What changes were proposed in this pull request?
This PR adds a new rule `PullOutGroupingExpressions` to pull out complex grouping expressions to a `Project` node under an `Aggregate`. These expressions are then referenced in both grouping expressions and aggregate expressions without aggregate functions to ensure that optimization rules don't change the aggregate expressions to invalid ones that no longer refer to any grouping expressions.

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

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

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

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

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

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

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

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-02 05:53:09 +00:00
ulysses-you 39889df32a [SPARK-35264][SQL] Support AQE side broadcastJoin threshold
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

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

### How was this patch tested?

Add new test.

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

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-30 09:16:21 +00:00
Kousuke Saruta e8bf8fe213 [SPARK-35047][SQL] Allow Json datasources to write non-ascii characters as codepoints
### What changes were proposed in this pull request?

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

New test.

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

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

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

### Why are the changes needed?

To make those commands more useful.

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

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

### How was this patch tested?

New tests.

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

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

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

### Why are the changes needed?

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

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

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

### How was this patch tested?

New test.

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

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

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

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

### Why are the changes needed?

fix ambiguity

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

no

### How was this patch tested?

not needed, just current CI pass

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

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-29 04:51:27 +00:00
Chao Sun 86d3bb5f7d [SPARK-34981][SQL] Implement V2 function resolution and evaluation
Co-Authored-By: Chao Sun <sunchaoapple.com>
Co-Authored-By: Ryan Blue <rbluenetflix.com>

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

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

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

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

### Why are the changes needed?

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

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

Yes, now a function exposed through V2 FunctionCatalog can be analyzed and evaluated.

### How was this patch tested?

Added new unit tests.

Closes #32082 from sunchao/resolve-func-v2.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Co-authored-by: Chao Sun <sunchao@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 17:21:49 +00:00
ulysses-you 0bcf348438 [SPARK-34781][SQL][FOLLOWUP] Adjust the order of AQE optimizer rules
### What changes were proposed in this pull request?

Reorder  `DemoteBroadcastHashJoin` and `EliminateUnnecessaryJoin`.

### Why are the changes needed?

Skip unnecessary check in `DemoteBroadcastHashJoin` if `EliminateUnnecessaryJoin` affects.

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

No

### How was this patch tested?

No result affect.

Closes #32380 from ulysses-you/SPARK-34781-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-28 13:59:24 +00:00
ulysses-you 8b62c2964d [SPARK-35214][SQL] OptimizeSkewedJoin support ShuffledHashJoinExec
### What changes were proposed in this pull request?

Add `ShuffledHashJoin` pattern check in `OptimizeSkewedJoin` so that we can optimize it.

### Why are the changes needed?

Currently, we have already supported all type of join through hint that make it easy to choose the join implementation.

We would choose `ShuffledHashJoin` if one table is not big but over the broadcast threshold. It's better that we can support optimize it in `OptimizeSkewedJoin`.

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

Probably yes, the execute plan in AQE mode may be changed.

### How was this patch tested?

Improve exists test in `AdaptiveQueryExecSuite`

Closes #32328 from ulysses-you/SPARK-35214.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-28 16:57:57 +09:00
PengLei 046c8c3dd6 [SPARK-34878][SQL][TESTS] Check actual sizes of year-month and day-time intervals
### What changes were proposed in this pull request?
As we have suport the year-month and day-time intervals.  Add the test actual size of year-month and day-time intervals type

### Why are the changes needed?
Just add test

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

### How was this patch tested?
./dev/scalastyle
run test for "ColumnTypeSuite"

Closes #32366 from Peng-Lei/SPARK-34878.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-28 07:48:49 +03:00
Kousuke Saruta abb1f0c5d7 [SPARK-35236][SQL] Support archive files as resources for CREATE FUNCTION USING syntax
### What changes were proposed in this pull request?

This PR proposes to make `CREATE FUNCTION USING` syntax can take archives as resources.

### Why are the changes needed?

It would be useful.
`CREATE FUNCTION USING` syntax doesn't support archives as resources because archives were not supported in Spark SQL.
Now Spark SQL supports archives so I think we can support them for the syntax.

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

Yes. Users can specify archives for `CREATE FUNCTION USING` syntax.

### How was this patch tested?

New test.

Closes #32359 from sarutak/load-function-using-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-28 10:15:21 +09:00
Kent Yao 16d223efee [SPARK-35091][SPARK-35090][SQL] Support extract from ANSI Intervals
### What changes were proposed in this pull request?

In this PR, we add extract/date_part support for ANSI Intervals

The `extract` is an ANSI expression and `date_part` is NON-ANSI but exists as an equivalence for `extract`

#### expression

```
<extract expression> ::=
  EXTRACT <left paren> <extract field> FROM <extract source> <right paren>
```

#### <extract field> for interval source

```

<primary datetime field> ::=
    <non-second primary datetime field>
| SECOND
<non-second primary datetime field> ::=
    YEAR
  | MONTH
  | DAY
  | HOUR
  | MINUTE
```

#### dataType

```
If <extract field> is a <primary datetime field> that does not specify SECOND or <extract field> is not a <primary datetime field>, then the declared type of the result is an implementation-defined exact numeric type with scale 0 (zero)

Otherwise, the declared type of the result is an implementation-defined exact numeric type with scale not less than the specified or implied <time fractional seconds precision> or <interval fractional seconds precision>, as appropriate, of the SECOND <primary datetime field> of the <extract source>.
```

### Why are the changes needed?

Subtask of ANSI Intervals Support

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

Yes
1. extract/date_part support ANSI intervals
2. for non-ansi intervals, the return type is changed from long to byte when extracting hours

### How was this patch tested?

new added tests

Closes #32351 from yaooqinn/SPARK-35091.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:06:54 +00:00
ulysses-you 4ff9f1fe3b [SPARK-35239][SQL] Coalesce shuffle partition should handle empty input RDD
### What changes were proposed in this pull request?

Create empty partition for custom shuffle reader if input RDD is empty.

### Why are the changes needed?

If input RDD partition is empty then the map output statistics will be null. And if all shuffle stage's input RDD partition is empty, we will skip it and lose the chance to coalesce partition.

We can simply create a empty partition for these custom shuffle reader to reduce the partition number.

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

Yes, the shuffle partition might be changed in AQE.

### How was this patch tested?

add new test.

Closes #32362 from ulysses-you/SPARK-35239.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 13:05:57 +00:00
gengjiaan 55dea2d937 [SPARK-34837][SQL][FOLLOWUP] Fix division by zero in the avg function over ANSI intervals
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/32229 support ANSI SQL intervals by the aggregate function `avg`.
But have not treat that the input zero rows. so this will lead to:
```
Caused by: java.lang.ArithmeticException: / by zero
	at com.google.common.math.LongMath.divide(LongMath.java:367)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1864)
	at org.apache.spark.rdd.RDD.$anonfun$count$1(RDD.scala:1253)
	at org.apache.spark.rdd.RDD.$anonfun$count$1$adapted(RDD.scala:1253)
	at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2248)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

### Why are the changes needed?
Fix a bug.

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

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

Closes #32358 from beliefer/SPARK-34837-followup.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-27 10:52:12 +03:00
Angerszhuuuu 2d2f467831 [SPARK-35169][SQL] Fix wrong result of min ANSI interval division by -1
### What changes were proposed in this pull request?
Before this patch
```
scala> Seq(java.time.Period.ofMonths(Int.MinValue)).toDF("i").select($"i" / -1).show(false)
+-------------------------------------+
|(i / -1)                             |
+-------------------------------------+
|INTERVAL '-178956970-8' YEAR TO MONTH|
+-------------------------------------+
scala> Seq(java.time.Duration.of(Long.MinValue, java.time.temporal.ChronoUnit.MICROS)).toDF("i").select($"i" / -1).show(false)
+---------------------------------------------------+
|(i / -1)                                           |
+---------------------------------------------------+
|INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND|
+---------------------------------------------------+
```

Wrong result of min ANSI interval division by -1, this pr fix this

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32314 from AngersZhuuuu/SPARK-35169.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 07:05:50 +00:00
Cheng Su c4ad86f311 [SPARK-35235][SQL][TEST] Add row-based hash map into aggregate benchmark
### What changes were proposed in this pull request?

`AggregateBenchmark` is only testing the performance for vectorized fast hash map, but not row-based hash map (which is used by default). We should add the row-based hash map into the benchmark.

java 8 benchmark run - https://github.com/c21/spark/actions/runs/787731549
java 11 benchmark run - https://github.com/c21/spark/actions/runs/787742858

### Why are the changes needed?

To have and track a basic sense of benchmarking different fast hash map used in hash aggregate.

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

No.

### How was this patch tested?

Existing unit test, as this only touches benchmark code.

Closes #32357 from c21/agg-benchmark.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 06:53:42 +00:00
PengLei eb08b9010a [SPARK-35139][SQL] Support ANSI intervals as Arrow Column vectors
### What changes were proposed in this pull request?
 Support YearMonthIntervalType and DayTimeIntervalType to extend ArrowColumnVector

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

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

### How was this patch tested?
1. By checking coding style via:
    $ ./dev/scalastyle
    $ ./dev/lint-java
2. Run the test "ArrowWriterSuite"

Closes #32340 from Peng-Lei/SPARK-35139.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 06:08:17 +00:00
Cheng Su 7f51106c0d [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table
### What changes were proposed in this pull request?

This is a re-proposal of https://github.com/apache/spark/pull/23163. Currently spark always requires a [local sort](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala#L188) before writing to output table with dynamic partition/bucket columns. The sort can be unnecessary if cardinality of partition/bucket values is small, and can be avoided by keeping multiple output writers concurrently.

This PR introduces a config `spark.sql.maxConcurrentOutputFileWriters` (which disables this feature by default), where user can tune the maximal number of concurrent writers. The config is needed here as we cannot keep arbitrary number of writers in task memory which can cause OOM (especially for Parquet/ORC vectorization writer).

The feature is to first use concurrent writers to write rows. If the number of writers exceeds the above config specified limit. Sort rest of rows and write rows one by one (See `DynamicPartitionDataConcurrentWriter.writeWithIterator()`).

In addition, interface `WriteTaskStatsTracker` and its implementation `BasicWriteTaskStatsTracker` are also changed because previously they are relying on the assumption that only one writer is active for writing dynamic partitions and bucketed table.

### Why are the changes needed?

Avoid the sort before writing output for dynamic partitioned query and bucketed table.
Help improve CPU and IO performance for these queries.

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

No.

### How was this patch tested?

Added unit test in `DataFrameReaderWriterSuite.scala`.

Closes #32198 from c21/writer.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-27 05:37:08 +00:00
Terry Kim 7779fce79a [SPARK-35225][SQL] EXPLAIN command should handle empty output of analyzed plan
### What changes were proposed in this pull request?

EXPLAIN command puts an empty line if there is no output for an analyzed plan. For example,

`sql("CREATE VIEW test AS SELECT 1").explain(true)` produces:
```
== Parsed Logical Plan ==
'CreateViewStatement [test], SELECT 1, false, false, PersistedView
+- 'Project [unresolvedalias(1, None)]
   +- OneRowRelation

== Analyzed Logical Plan ==

CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation

== Optimized Logical Plan ==
CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation

== Physical Plan ==
Execute CreateViewCommand
   +- CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
         +- Project [1 AS 1#7]
            +- OneRowRelation
```

### Why are the changes needed?

To handle empty output of analyzed plan and remove the unneeded empty line.

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

Yes, now the EXPLAIN command for the analyzed plan produces the following without the empty line:
```
== Analyzed Logical Plan ==
CreateViewCommand `default`.`test`, SELECT 1, false, false, PersistedView, true
   +- Project [1 AS 1#7]
      +- OneRowRelation
```

### How was this patch tested?

Added a test.

Closes #32342 from imback82/analyzed_plan_blank_line.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-27 11:10:16 +09:00
Shixiong Zhu 0df3b501ae [SPARK-28247][SS][TEST] Fix flaky test "query without test harness" on ContinuousSuite
### What changes were proposed in this pull request?

This is another attempt to fix the flaky test "query without test harness" on ContinuousSuite.

`query without test harness` is flaky because it starts a continuous query with two partitions but assumes they will run at the same speed.

In this test, 0 and 2 will be written to partition 0, 1 and 3 will be written to partition 1. It assumes when we see 3, 2 should be written to the memory sink. But this is not guaranteed. We can add `if (currentValue == 2) Thread.sleep(5000)` at this line b2a2b5d820/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala (L135) to reproduce the failure: `Result set Set([0], [1], [3]) are not a superset of Set(0, 1, 2, 3)!`

The fix is changing `waitForRateSourceCommittedValue` to wait until all partitions reach the desired values before stopping the query.

### Why are the changes needed?

Fix a flaky test.

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

No

### How was this patch tested?

Existing tests. Manually verify the reproduction I mentioned above doesn't fail after this change.

Closes #32316 from zsxwing/SPARK-28247-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-04-27 08:07:09 +09:00
Angerszhuuuu f0090463a8 [SPARK-33985][SQL][TESTS] Add query test of combine usage of TRANSFORM and CLUSTER BY/ORDER BY
### What changes were proposed in this pull request?
Under hive's document  https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Transform there are many usage about  TRANSFORM and CLUSTER BY/ORDER BY, in this pr add some test about this cases.

### Why are the changes needed?
Add UT

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

### How was this patch tested?
Added UT

Closes #32333 from AngersZhuuuu/SPARK-33985.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-26 16:42:07 +00:00
Liang-Chi Hsieh c59988aa79 [SPARK-34638][SQL] Single field nested column prune on generator output
### What changes were proposed in this pull request?

This patch proposes an improvement on nested column pruning if the pruning target is generator's output. Previously we disallow such case. This patch allows to prune on it if there is only one single nested column is accessed after `Generate`.

E.g., `df.select(explode($"items").as('item)).select($"item.itemId")`. As we only need `itemId` from `item`, we can prune other fields out and only keep `itemId`.

In this patch, we only address explode-like generators. We will address other generators in followups.

### Why are the changes needed?

This helps to extend the availability of nested column pruning.

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

No

### How was this patch tested?

Unit test

Closes #31966 from viirya/SPARK-34638.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-26 09:32:22 -07:00
Liang-Chi Hsieh bdac19184a [SPARK-35230][SQL] Move custom metric classes to proper package
### What changes were proposed in this pull request?

This patch moves DS v2 custom metric classes to `org.apache.spark.sql.connector.metric` package. Moving `CustomAvgMetric` and `CustomSumMetric` to above package and make them as public java abstract class too.

### Why are the changes needed?

`CustomAvgMetric` and `CustomSumMetric`  should be public APIs for developers to extend. As there are a few metric classes, we should put them together in one package.

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

No, dev only and they are not released yet.

### How was this patch tested?

Unit tests.

Closes #32348 from viirya/move-custom-metric-classes.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-26 07:19:36 -07:00
Adam Binford 74afc68e21 [SPARK-35213][SQL] Keep the correct ordering of nested structs in chained withField operations
### What changes were proposed in this pull request?

Modifies the UpdateFields optimizer to fix correctness issues with certain nested and chained withField operations. Examples for recreating the issue are in the new unit tests as well as the JIRA issue.

### Why are the changes needed?

Certain withField patterns can cause Exceptions or even incorrect results. It appears to be a result of the additional UpdateFields optimization added in https://github.com/apache/spark/pull/29812. It traverses fieldOps in reverse order to take the last one per field, but this can cause nested structs to change order which leads to mismatches between the schema and the actual data. This updates the optimization to maintain the initial ordering of nested structs to match the generated schema.

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

It fixes exceptions and incorrect results for valid uses in the latest Spark release.

### How was this patch tested?

Added new unit tests for these edge cases.

Closes #32338 from Kimahriman/bug/optimize-with-fields.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-25 23:39:56 -07:00
Angerszhuuuu 6f782efb04 [SPARK-35220][SQL] DayTimeIntervalType/YearMonthIntervalType show different between Hive SerDe and row format delimited
### What changes were proposed in this pull request?
DayTimeIntervalType/YearMonthIntervalString show different between Hive SerDe and row format delimited.
Create this pr to add a test and  have disscuss.

For this problem I think we have two direction:

1. leave it as current and add a item t explain this  in migration guide docs.
2. Since we should not change hive serde's behavior, so we can cast spark row format delimited's behavior to use cast  DayTimeIntervalType/YearMonthIntervalType as HIVE_STYLE

### Why are the changes needed?
Add UT

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

### How was this patch tested?
added ut

Closes #32335 from AngersZhuuuu/SPARK-35220.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-26 11:26:32 +09:00
Kent Yao 5b1353f690 [SPARK-35168][SQL] mapred.reduce.tasks should be shuffle.partitions not adaptive.coalescePartitions.initialPartitionNum
### What changes were proposed in this pull request?

```sql
spark-sql> set spark.sql.adaptive.coalescePartitions.initialPartitionNum=1;
spark.sql.adaptive.coalescePartitions.initialPartitionNum	1
Time taken: 2.18 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks;
21/04/21 14:27:11 WARN SetCommand: Property mapred.reduce.tasks is deprecated, showing spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	1
Time taken: 0.03 seconds, Fetched 1 row(s)
spark-sql> set spark.sql.shuffle.partitions;
spark.sql.shuffle.partitions	200
Time taken: 0.024 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks=2;
21/04/21 14:31:52 WARN SetCommand: Property mapred.reduce.tasks is deprecated, automatically converted to spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	2
Time taken: 0.017 seconds, Fetched 1 row(s)
spark-sql> set mapred.reduce.tasks;
21/04/21 14:31:55 WARN SetCommand: Property mapred.reduce.tasks is deprecated, showing spark.sql.shuffle.partitions instead.
spark.sql.shuffle.partitions	1
Time taken: 0.017 seconds, Fetched 1 row(s)
spark-sql>
```

`mapred.reduce.tasks` is mapping to `spark.sql.shuffle.partitions` at write-side, but `spark.sql.adaptive.coalescePartitions.initialPartitionNum` might take precede of `spark.sql.shuffle.partitions`

### Why are the changes needed?

roundtrip for `mapred.reduce.tasks`

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

yes, `mapred.reduce.tasks` will always report `spark.sql.shuffle.partitions` whether `spark.sql.adaptive.coalescePartitions.initialPartitionNum` exists or not.

### How was this patch tested?

a new test

Closes #32265 from yaooqinn/SPARK-35168.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-25 20:27:12 +08:00
Liang-Chi Hsieh b2a2b5d820 [SPARK-34297][SQL][SS] Add metrics for data loss and offset out range for KafkaMicroBatchStream
### What changes were proposed in this pull request?

This patch proposes to add a couple of metrics in scan node for Kafka batch streaming query.

### Why are the changes needed?

When testing SS, I found it is hard to track data loss of SS reading from Kafka. The micro batch scan node has only one metric, number of output rows. Users have no idea how many offsets to fetch are out of Kafka, how many times data loss happens. These metrics are important for users to know the quality of SS query running.

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

Yes, adding two metrics to micro batch scan node for Kafka batch streaming.

### How was this patch tested?

Currently I tested on internal cluster with Kafka:

<img width="1193" alt="Screen Shot 2021-04-22 at 7 16 29 PM" src="https://user-images.githubusercontent.com/68855/115808460-61bf8100-a39f-11eb-99a9-65d22c3f5fb0.png">

I was trying to add unit test. But as our batch streaming query disallows to specify ending offsets. If I only specify an out-of-range starting offset, when we get offset range in `getRanges`,  any negative size range will be filtered out. So it cannot actually test the case of fetched non-existing offset.

Closes #31398 from viirya/micro-batch-metrics.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-23 13:56:53 -07:00
Wenchen Fan fdccd88c2a Revert "[SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function"
This reverts commit c8d78a70b4.
2021-04-23 15:55:30 +08:00
Cheng Su cab205e9e4 [SPARK-35141][SQL] Support two level of hash maps for final hash aggregation
### What changes were proposed in this pull request?

For partial hash aggregation (code-gen path), we have two level of hash map for aggregation. First level is from `RowBasedHashMapGenerator`, which is computation faster compared to the second level from `UnsafeFixedWidthAggregationMap`. The introducing of two level hash map can help improve CPU performance of query as the first level hash map normally fits in hardware cache and has cheaper hash function for key lookup.

For final hash aggregation, we can also support two level of hash map, to improve query performance further.
The original two level of hash map code works for final aggregation mostly out of box. The major change here is to support testing fall back of final aggregation (see change related to `bitMaxCapacity` and `checkFallbackForGeneratedHashMap`).

Example:

An aggregation query:

```
spark.sql(
  """
    |SELECT key, avg(value)
    |FROM agg1
    |GROUP BY key
  """.stripMargin)
```

The generated code for final aggregation is [here](https://gist.github.com/c21/20c10cc8e2c7e561aafbe9b8da055242).

An aggregation query with testing fallback:
```
withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> "2, 3") {
  spark.sql(
    """
      |SELECT key, avg(value)
      |FROM agg1
      |GROUP BY key
    """.stripMargin)
}
```
The generated code for final aggregation is [here](https://gist.github.com/c21/dabf176cbc18a5e2138bc0a29e81c878). Note the no more counter condition for first level fast map.

### Why are the changes needed?

Improve the CPU performance of hash aggregation query in general.

For `AggregateBenchmark."Aggregate w multiple keys"`, seeing query performance improved by 10%.
`codegen = T` means whole stage code-gen is enabled.
`hashmap = T` means two level maps is enabled for partial aggregation.
`finalhashmap = T` means two level maps is enabled for final aggregation.

```
Running benchmark: Aggregate w multiple keys
  Running case: codegen = F
  Stopped after 2 iterations, 8284 ms
  Running case: codegen = T hashmap = F
  Stopped after 2 iterations, 5424 ms
  Running case: codegen = T hashmap = T finalhashmap = F
  Stopped after 2 iterations, 4753 ms
  Running case: codegen = T hashmap = T finalhashmap = T
  Stopped after 2 iterations, 4508 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Aggregate w multiple keys:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
codegen = F                                        3881           4142         370          5.4         185.1       1.0X
codegen = T hashmap = F                            2701           2712          16          7.8         128.8       1.4X
codegen = T hashmap = T finalhashmap = F           2363           2377          19          8.9         112.7       1.6X
codegen = T hashmap = T finalhashmap = T           2252           2254           3          9.3         107.4       1.7X
```

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

No.

### How was this patch tested?

Existing unit test in `HashAggregationQuerySuite` and `HashAggregationQueryWithControlledFallbackSuite` already cover the test.

Closes #32242 from c21/agg.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-23 05:24:19 +00:00
Chao Sun 86238d0e88 [SPARK-35195][SQL][TEST] Move InMemoryTable etc to org.apache.spark.sql.connector.catalog
### What changes were proposed in this pull request?

Move the following classes:
- `InMemoryAtomicPartitionTable`
- `InMemoryPartitionTable`
- `InMemoryPartitionTableCatalog`
- `InMemoryTable`
- `InMemoryTableCatalog`
- `StagingInMemoryTableCatalog`

from `org.apache.spark.sql.connector` to `org.apache.spark.sql.connector.catalog`.

### Why are the changes needed?

These classes implement catalog related interfaces but reside in `org.apache.spark.sql.connector`. A more suitable place should be `org.apache.spark.sql.connector.catalog`.

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

No

### How was this patch tested?

N/A

Closes #32302 from sunchao/SPARK-35195.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-22 21:56:45 -07:00
Yingyi Bu 47f86875f7 [SPARK-35075][SQL] Add traversal pruning for subquery related rules
### What changes were proposed in this pull request?

Added the following TreePattern enums:
- DYNAMIC_PRUNING_SUBQUERY
- EXISTS_SUBQUERY
- IN_SUBQUERY
- LIST_SUBQUERY
- PLAN_EXPRESSION
- SCALAR_SUBQUERY
- FILTER

Used them in the following rules:
- ResolveSubquery
- UpdateOuterReferences
- OptimizeSubqueries
- RewritePredicateSubquery
- PullupCorrelatedPredicates
- RewriteCorrelatedScalarSubquery (not the rule itself but an internal transform call, the full support is in SPARK-35148)
- InsertAdaptiveSparkPlan
- PlanAdaptiveSubqueries

### Why are the changes needed?

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

### How was this patch tested?

Existing tests.

Closes #32247 from sigmod/subquery.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-23 12:42:55 +08:00
Angerszhuuuu 04e2305a9b [SPARK-35187][SQL] Fix failure on the minimal interval literal
### What changes were proposed in this pull request?
If the sign '-' inside of interval string, everything is fine after bb5459fb26:
```
spark-sql> SELECT INTERVAL '-178956970-8' YEAR TO MONTH;
-178956970-8
```
but the sign outside of interval string is not handled properly:
```
spark-sql> SELECT INTERVAL -'178956970-8' YEAR TO MONTH;
Error in query:
Error parsing interval year-month string: integer overflow(line 1, pos 16)

== SQL ==
SELECT INTERVAL -'178956970-8' YEAR TO MONTH
----------------^^^
```
This pr fix this issue

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32296 from AngersZhuuuu/SPARK-35187.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-22 21:58:57 +03:00
beliefer 6c587d2627 [SPARK-35110][SQL] Handle ANSI intervals in WindowExecBase
### What changes were proposed in this pull request?
This PR makes window frame could support `YearMonthIntervalType` and `DayTimeIntervalType`.

### Why are the changes needed?
Extend the function of window frame

### Does this PR introduce _any_ user-facing change?
Yes. Users could use `YearMonthIntervalType` or `DayTimeIntervalType` as the sort expression for window frame.

### How was this patch tested?
New tests

Closes #32294 from beliefer/SPARK-35110.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-22 17:14:11 +03:00
Angerszhuuuu b22d54a58a [SPARK-35026][SQL] Support nested CUBE/ROLLUP/GROUPING SETS in GROUPING SETS
### What changes were proposed in this pull request?
PG and Oracle both support use CUBE/ROLLUP/GROUPING SETS in GROUPING SETS's grouping set as a sugar syntax.
![image](https://user-images.githubusercontent.com/46485123/114975588-139a1180-9eb7-11eb-8f53-498c1db934e0.png)

In this PR, we support it in Spark SQL too

### Why are the changes needed?
Keep consistent with PG and oracle

### Does this PR introduce _any_ user-facing change?
User can write grouping analytics like
```
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b));
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ());
SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()));
```

### How was this patch tested?
Added Test

Closes #32201 from AngersZhuuuu/SPARK-35026.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-22 13:08:22 +00:00
Yuming Wang e609395913 [SPARK-34897][SQL] Support reconcile schemas based on index after nested column pruning
### What changes were proposed in this pull request?

It will remove `StructField` when [pruning nested columns](0f2c0b53e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala (L28-L42)). For example:
```scala
spark.sql(
  """
    |CREATE TABLE t1 (
    |  _col0 INT,
    |  _col1 STRING,
    |  _col2 STRUCT<c1: STRING, c2: STRING, c3: STRING, c4: BIGINT>)
    |USING ORC
    |""".stripMargin)

spark.sql("INSERT INTO t1 values(1, '2', struct('a', 'b', 'c', 10L))")

spark.sql("SELECT _col0, _col2.c1 FROM t1").show
```

Before this pr. The returned schema is: ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` add it will throw exception:
```
java.lang.AssertionError: assertion failed: The given data schema struct<_col0:int,_col2:struct<c1:string>> has less fields than the actual ORC physical schema, no idea which columns were dropped, fail to read.
	at scala.Predef$.assert(Predef.scala:223)
	at org.apache.spark.sql.execution.datasources.orc.OrcUtils$.requestedColumnIds(OrcUtils.scala:160)
```

After this pr. The returned schema is: ``` `_col0` INT,`_col1` STRING,`_col2` STRUCT<`c1`: STRING> ```.

The finally schema is ``` `_col0` INT,`_col2` STRUCT<`c1`: STRING> ``` after the complete column pruning:
7a5647a93a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala (L208-L213)

e64eb75aed/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala (L96-L97)

### Why are the changes needed?

Fix bug.

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

No.

### How was this patch tested?

Unit test.

Closes #31993 from wangyum/SPARK-34897.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-21 10:23:38 -07:00
ulysses-you 81dbaedede [SPARK-34692][SQL] Support Not(Int) and Not(InSet) propagate null in predicate
### What changes were proposed in this pull request?

* Add `Not(In)` and `Not(InSet)` check in `NullPropagation` rule.
* Add more test for `In` and `Not(In)` in `Project` level.

### Why are the changes needed?

The semantics of `Not(In)` could be seen like `And(a != b, a != c)` that match the `NullIntolerant`.

As we already simplify the `NullIntolerant` expression to null if it's children have null. E.g. `a != null` => `null`. It's safe to do this with `Not(In)`/`Not(InSet)`.

Note that, we can only do the simplify in predicate which `ReplaceNullWithFalseInPredicate`  rule do.

Let's say we have two sqls:
```
select 1 not in (2, null);
select 1 where 1 not in (2, null);
```
The first sql we cannot optimize since it would return `NULL` instead of `false`. The second is postive.

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

No.

### How was this patch tested?

Add test.

Closes #31797 from ulysses-you/SPARK-34692.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-21 15:37:02 +00:00
Kent Yao 81c3cc2312 [SPARK-35044][SQL][FOLLOWUP][TEST-HADOOP2.7] Fix hadoop 2.7 test due to diff between hadoop 2.7 and hadoop 3
### What changes were proposed in this pull request?

dfs.replication is inconsistent from hadoop 2.x to 3.x, so in this PR we use `dfs.hosts` to verify per https://github.com/apache/spark/pull/32144#discussion_r616833099

```
== Results ==
!== Correct Answer - 1 ==        == Spark Answer - 1 ==
!struct<>                        struct<key:string,value:string>
![dfs.replication,<undefined>]   [dfs.replication,3]
```

### Why are the changes needed?

fix Jenkins job with Hadoop 2.7

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

test only change
### How was this patch tested?

test only change

Closes #32263 from yaooqinn/SPARK-35044-F.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-21 09:10:05 +09:00
Max Gekk e8d6992b66 [SPARK-35153][SQL] Make textual representation of ANSI interval operators more readable
### What changes were proposed in this pull request?
In the PR, I propose to override the `sql` and `toString` methods of the expressions that implement operators over ANSI intervals (`YearMonthIntervalType`/`DayTimeIntervalType`), and replace internal expression class names by operators like `*`, `/` and `-`.

### Why are the changes needed?
Proposed methods should make the textual representation of such operators more readable, and potentially parsable by Spark SQL parser.

### Does this PR introduce _any_ user-facing change?
Yes. This can influence on column names.

### How was this patch tested?
By running existing test suites for interval and datetime expressions, and re-generating the `*.sql` tests:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
```

Closes #32262 from MaxGekk/interval-operator-sql.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-20 23:13:40 +03:00
Wenchen Fan e08c40fa3f [SPARK-35145][SQL] CurrentOrigin should support nested invoking
### What changes were proposed in this pull request?

`CurrentOrigin` is a thread-local variable to track the original SQL line position in plan/expression. Usually, we set `CurrentOrigin`, create `TreeNode` instances, and reset `CurrentOrigin`.

This PR updates the last step to set `CurrentOrigin` to its previous value, instead of resetting it. This is necessary when we invoke `CurrentOrigin` in a nested way, like with subqueries.

### Why are the changes needed?

To keep the original SQL line position in the error message in more cases.

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

No, only minor error message changes.

### How was this patch tested?

existing tests

Closes #32249 from cloud-fan/origin.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 15:23:41 +00:00
Liang-Chi Hsieh eb9a4390da [SPARK-34338][SQL] Report metrics from Datasource v2 scan
### What changes were proposed in this pull request?

This patch proposes to leverage `CustomMetric`, `CustomTaskMetric` API to report custom metrics from DS v2 scan to Spark.

### Why are the changes needed?

This is related to #31398. In SPARK-34297, we want to add a couple of metrics when reading from Kafka in SS. We need some public API change in DS v2 to make it possible. This extracts only DS v2 change and make it general for DS v2 instead of micro-batch DS v2 API.

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

No

### How was this patch tested?

Unit test.

Implement a simple test DS v2 class locally and run it:

```scala
scala> import org.apache.spark.sql.execution.datasources.v2._
import org.apache.spark.sql.execution.datasources.v2._

scala> classOf[CustomMetricDataSourceV2].getName
res0: String = org.apache.spark.sql.execution.datasources.v2.CustomMetricDataSourceV2

scala> val df = spark.read.format(res0).load()
df: org.apache.spark.sql.DataFrame = [i: int, j: int]

scala> df.collect
```

<img width="703" alt="Screen Shot 2021-03-30 at 11 07 13 PM" src="https://user-images.githubusercontent.com/68855/113098080-d8a49800-91ac-11eb-8681-be408a0f2e69.png">

Closes #31451 from viirya/dsv2-metrics.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 15:01:44 +00:00
Angerszhuuuu 361444890e [SPARK-34035][SQL] Refactor ScriptTransformation to remove input parameter and replace it by child.output
### What changes were proposed in this pull request?
Refactor ScriptTransformation to remove input parameter and replace it by child.output

### Why are the changes needed?
refactor code

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

### How was this patch tested?
Existed UT

Closes #32228 from AngersZhuuuu/SPARK-34035.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 14:52:21 +00:00
allisonwang-db bad4b6f025 [SPARK-35080][SQL] Only allow a subset of correlated equality predicates when a subquery is aggregated
### What changes were proposed in this pull request?
This PR updated the `foundNonEqualCorrelatedPred` logic for correlated subqueries in `CheckAnalysis` to only allow correlated equality predicates that guarantee one-to-one mapping between inner and outer attributes, instead of all equality predicates.

### Why are the changes needed?
To fix correctness bugs. Before this fix Spark can give wrong results for certain correlated subqueries that pass CheckAnalysis:
Example 1:
```sql
create or replace view t1(c) as values ('a'), ('b')
create or replace view t2(c) as values ('ab'), ('abc'), ('bc')

select c, (select count(*) from t2 where t1.c = substring(t2.c, 1, 1)) from t1
```
Correct results: [(a, 2), (b, 1)]
Spark results:
```
+---+-----------------+
|c  |scalarsubquery(c)|
+---+-----------------+
|a  |1                |
|a  |1                |
|b  |1                |
+---+-----------------+
```
Example 2:
```sql
create or replace view t1(a, b) as values (0, 6), (1, 5), (2, 4), (3, 3);
create or replace view t2(c) as values (6);

select c, (select count(*) from t1 where a + b = c) from t2;
```
Correct results: [(6, 4)]
Spark results:
```
+---+-----------------+
|c  |scalarsubquery(c)|
+---+-----------------+
|6  |1                |
|6  |1                |
|6  |1                |
|6  |1                |
+---+-----------------+
```
### Does this PR introduce _any_ user-facing change?
Yes. Users will not be able to run queries that contain unsupported correlated equality predicates.

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

Closes #32179 from allisonwang-db/spark-35080-subquery-bug.

Lead-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-20 11:11:40 +08:00
Yingyi Bu 9a6d7730f5 [SPARK-35103][SQL] Make TypeCoercion rules more efficient
## What changes were proposed in this pull request?
This PR fixes a couple of things in TypeCoercion rules:
- Only run the propagate types step if the children of a node have output attributes with changed dataTypes and/or nullability. This is implemented as custom tree transformation. The TypeCoercion rules now only implement a partial function.
- Combine multiple type coercion rules into a single rule. Multiple rules are applied in single tree traversal.
- Reduce calls to conf.get in DecimalPrecision. This now happens once per tree traversal, instead of once per matched expression.
- Reduce the use of withNewChildren.

This brings down the number of CPU cycles spend in analysis by ~28% (benchmark: 10 iterations of all TPC-DS queries on SF10).

## How was this patch tested?
Existing tests.

Closes #32208 from sigmod/coercion.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-19 21:25:58 +02:00
Max Gekk 1d1ed3eb25 [SPARK-35107][SQL] Parse unit-to-unit interval literals to ANSI intervals
### What changes were proposed in this pull request?
Parse the year-month interval literals like `INTERVAL '1-1' YEAR TO MONTH` to values of `YearMonthIntervalType`, and day-time interval literals to `DayTimeIntervalType` values. Currently, Spark SQL supports:
- DAY TO HOUR
- DAY TO MINUTE
- DAY TO SECOND
- HOUR TO MINUTE
- HOUR TO SECOND
- MINUTE TO SECOND

All such interval literals are converted to `DayTimeIntervalType`, and `YEAR TO MONTH` to `YearMonthIntervalType` while loosing info about `from` and `to` units.

**Note**: new behavior is under the SQL config `spark.sql.legacy.interval.enabled` which is `false` by default. When the config is set to `true`, the interval literals are parsed to `CaledarIntervalType` values.

Closes #32176

### Why are the changes needed?
To conform the ANSI SQL standard which assumes conversions of interval literals to year-month or day-time interval but not to mixed interval type like Catalyst's `CalendarIntervalType`.

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

Before:
```sql
spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND;
1 days 1 hours 2 minutes 3.123 seconds
spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND);
interval
```

After:
```sql
spark-sql> SELECT INTERVAL '1 01:02:03.123' DAY TO SECOND;
1 01:02:03.123000000
spark-sql> SELECT typeof(INTERVAL '1 01:02:03.123' DAY TO SECOND);
day-time interval
```

### How was this patch tested?
1. By running the affected test suites:
```
$ ./build/sbt "test:testOnly *.ExpressionParserSuite"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z create_view.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
```
2. PostgresSQL tests are executed with `spark.sql.legacy.interval.enabled` is set to `true` to keep compatibility with PostgreSQL output:
```sql
> SELECT interval '999' second;
0 years 0 mons 0 days 0 hours 16 mins 39.00 secs
```

Closes #32209 from MaxGekk/parse-ansi-interval-literals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 16:00:59 +03:00
gengjiaan 8dc455bba8 [SPARK-34837][SQL] Support ANSI SQL intervals by the aggregate function avg
### What changes were proposed in this pull request?
Extend the `Average` expression to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the overflow exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
Extend `org.apache.spark.sql.catalyst.expressions.aggregate.Average` to support `DayTimeIntervalType` and `YearMonthIntervalType`.

### Does this PR introduce _any_ user-facing change?
'No'.
Should not since new types have not been released yet.

### How was this patch tested?
Jenkins test

Closes #32229 from beliefer/SPARK-34837.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-19 15:56:56 +03:00
Angerszhuuuu a74f601040 [SPARK-31937][SQL] Support processing ArrayType/MapType/StructType data using no-serde mode script transform
### What changes were proposed in this pull request?
Support no-serde mode script transform use ArrayType/MapType/StructStpe data.

### Why are the changes needed?
Make user can process array/map/struct data

### Does this PR introduce _any_ user-facing change?
Yes, user can process array/map/struct data in script transform `no-serde` mode

### How was this patch tested?
Added UT

Closes #30957 from AngersZhuuuu/SPARK-31937.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-19 17:02:32 +09:00
Terry Kim 7a06cdd53b [SPARK-35122][SQL] Migrate CACHE/UNCACHE TABLE to use AnalysisOnlyCommand
### What changes were proposed in this pull request?

Now that `AnalysisOnlyCommand` in introduced in #32032, `CacheTable` and `UncacheTable` can extend `AnalysisOnlyCommand` to simplify the code base. For example, the logic to handle these commands such that the tables are only analyzed is scattered across different places.

### Why are the changes needed?

To simplify the code base to handle these two commands.

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

No, just internal refactoring.

### How was this patch tested?

The existing tests (e.g., `CachedTableSuite`) cover the changes in this PR. For example, if I make `CacheTable`/`UncacheTable` extend `LeafCommand`, there are few failures in `CachedTableSuite`.

Closes #32220 from imback82/cache_cmd_analysis_only.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 06:00:23 +00:00
Peter Toth c8d78a70b4 [SPARK-34581][SQL] Don't optimize out grouping expressions from aggregate expressions without aggregate function
### What changes were proposed in this pull request?
This PR:
- Adds a new expression `GroupingExprRef` that can be used in aggregate expressions of `Aggregate` nodes to refer grouping expressions by index. These expressions capture the data type and nullability of the referred grouping expression.
- Adds a new rule `EnforceGroupingReferencesInAggregates` that inserts the references in the beginning of the optimization phase.
- Adds a new rule `UpdateGroupingExprRefNullability` to update nullability of `GroupingExprRef` expressions as nullability of referred grouping expression can change during optimization.

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

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

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

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

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

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

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

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 04:58:41 +00:00
Cheng Su fd08c93151 [SPARK-35109][SQL] Fix minor exception messages of HashedRelation and HashJoin
### What changes were proposed in this pull request?

It seems that we miss classifying one `SparkOutOfMemoryError` in `HashedRelation`. Add the error classification for it. In addition, clean up two errors definition of `HashJoin` as they are not used.

### Why are the changes needed?

Better error classification.

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

No.

### How was this patch tested?

Existing tests.

Closes #32211 from c21/error-message.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-19 12:43:43 +09:00
gengjiaan 12abfe7917 [SPARK-34716][SQL] Support ANSI SQL intervals by the aggregate function sum
### What changes were proposed in this pull request?
Extend the `Sum` expression to  to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the overflow exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
Extend `org.apache.spark.sql.catalyst.expressions.aggregate.Sum` to support `DayTimeIntervalType` and `YearMonthIntervalType`.

### Does this PR introduce _any_ user-facing change?
'No'.
Should not since new types have not been released yet.

### How was this patch tested?
Jenkins test

Closes #32107 from beliefer/SPARK-34716.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-18 18:03:50 +03:00
Kousuke Saruta 95db7e6459 [SPARK-35104][SQL] Fix ugly indentation of multiple JSON records in a single split file generated by JacksonGenerator when pretty option is true
### What changes were proposed in this pull request?

This issue fixes an issue that indentation of multiple output JSON records in a single split file are broken except for the first record in the split when `pretty` option is `true`.
```
// Run in the Spark Shell.
// Set spark.sql.leafNodeDefaultParallelism to 1 for the current master.
// Or set spark.default.parallelism for the previous releases.
spark.conf.set("spark.sql.leafNodeDefaultParallelism", 1)
val df = Seq("a", "b", "c").toDF
df.write.option("pretty", "true").json("/path/to/output")

# Run in a Shell
$ cat /path/to/output/*.json
{
  "value" : "a"
}
 {
  "value" : "b"
}
 {
  "value" : "c"
}
```

### Why are the changes needed?

It's not pretty even though `pretty` option is true.

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

I think "No". Indentation style is changed but JSON format is not changed.

### How was this patch tested?

New test.

Closes #32203 from sarutak/fix-ugly-indentation.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-16 11:00:52 +03:00
Angerszhuuuu 71133e1c2a [SPARK-35070][SQL] TRANSFORM not support alias in inputs
### What changes were proposed in this pull request?
Normal function parameters should not support alias, hive not support too
![image](https://user-images.githubusercontent.com/46485123/114645556-4a7ff400-9d0c-11eb-91eb-bc679ea0039a.png)
In this pr we forbid use alias in `TRANSFORM`'s inputs

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32165 from AngersZhuuuu/SPARK-35070.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-15 06:51:07 +00:00
Kent Yao f32114d17e [SPARK-35044][SQL] SET propertyKey shall also lookup sparkSession.sharedState.hadoopConf to display the effective default hive/hadoop configs
### What changes were proposed in this pull request?

Currently, pure SQL users are short of ways to see the Hadoop configurations which may affect their jobs a lot, they are only able to get the Hadoop configs that exist in `SQLConf` while other defaults in `SharedState.hadoopConf` display wrongly and confusingly with `<undefined>`.

The pre-loaded ones from `core-site.xml, hive-site.xml` etc., will only stay in `sparkSession.sharedState.hadoopConf` or `sc._hadoopConfiguation` not `SQLConf`. Some of them that related the Hive Metastore connection(never change it spark runtime), e.g. `hive.metastore.uris`, are clearly global static and unchangeable but displayable I guess. Some of the ones that might be related to, for example, the output codec/compression, preset in Hadoop/hive config files like core-site.xml shall be still changeable from case to case, table to table, file to file, etc. It' meaningfully to show the defaults for users to change based on that.

In this PR, I propose to support get a Hadoop configuration by SET syntax, for example
```
SET mapreduce.map.output.compress.codec;
```

### Why are the changes needed?

better user experience for pure SQL users

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

yes, where retrieving a conf only existing in sessionState.hadoopConf, before is `undefined` and now you see it

### How was this patch tested?

new test

Closes #32144 from yaooqinn/SPARK-35044.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-15 01:44:10 +08:00
Max Gekk de9e8b6c94 [SPARK-35051][SQL] Support add/subtract of a day-time interval to/from a date
### What changes were proposed in this pull request?
Support `date +/- day-time interval`. In the PR, I propose to update the binary arithmetic rules, and cast an input date to a timestamp at the session time zone, and then add a day-time interval to it.

### Why are the changes needed?
1. To conform the ANSI SQL standard which requires to support such operation over dates and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/111081674-865d4900-8515-11eb-86c8-3538ecaf4804.png">
2. To fix the regression comparing to the recent Spark release 3.1 with default settings.

Before the changes:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
Error in query: cannot resolve 'DATE '2021-04-14' + subtracttimestamps(TIMESTAMP '2021-04-14 18:14:56.497', TIMESTAMP '2021-04-13 00:00:00')' due to data type mismatch: argument 1 requires timestamp type, however, 'DATE '2021-04-14'' is of date type.; line 1 pos 7;
'Project [unresolvedalias(cast(2021-04-14 + subtracttimestamps(2021-04-14 18:14:56.497, 2021-04-13 00:00:00, false, Some(Europe/Moscow)) as date), None)]
+- OneRowRelation
```

Spark 3.1:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
2021-04-15
```

Hive:
```sql
0: jdbc:hive2://localhost:10000/default> select date'2021-04-14' + (timestamp'2020-04-14 18:15:30' - timestamp'2020-04-13 00:00:00');
+------------------------+
|          _c0           |
+------------------------+
| 2021-04-15 18:15:30.0  |
+------------------------+
```

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

After the changes:
```sql
spark-sql> select date'now' + (timestamp'now' - timestamp'yesterday');
2021-04-15 18:13:16.555
```

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #32170 from MaxGekk/date-add-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-14 19:28:26 +03:00
Angerszhuuuu 4ca9958270 [SPARK-35069][SQL] TRANSFORM forbidden DISTICNT and ALL, also make the error clear
### What changes were proposed in this pull request?
According to https://github.com/apache/spark/pull/29087#discussion_r612267050,  add UT in `transform.sql`

It seems that distinct is not recognized as a reserved word here

```
-- !query
explain extended SELECT TRANSFORM(distinct b, a, c)
                   USING 'cat' AS (a, b, c)
                 FROM script_trans
                 WHERE a <= 4
-- !query schema
struct<plan:string>
-- !query output
== Parsed Logical Plan ==
'ScriptTransformation [*], cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false)
+- 'Project ['distinct AS b#x, 'a, 'c]
   +- 'Filter ('a <= 4)
      +- 'UnresolvedRelation [script_trans], [], false

== Analyzed Logical Plan ==
org.apache.spark.sql.AnalysisException: cannot resolve 'distinct' given input columns: [script_trans.a, script_trans.b, script_trans.c]; line 1 pos 34;
'ScriptTransformation [*], cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false)
+- 'Project ['distinct AS b#x, a#x, c#x]
   +- Filter (a#x <= 4)
      +- SubqueryAlias script_trans
         +- View (`script_trans`, [a#x,b#x,c#x])
            +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x]
               +- Project [a#x, b#x, c#x]
                  +- SubqueryAlias script_trans
                     +- LocalRelation [a#x, b#x, c#x]
```

Hive's error
![image](https://user-images.githubusercontent.com/46485123/114533170-355d8380-9c80-11eb-992f-982f0b296759.png)

### Why are the changes needed?

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

### How was this patch tested?
Added Ut

Closes #32149 from AngersZhuuuu/SPARK-28227-new-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 15:03:29 +00:00
Terry Kim b5241c97b1 [SPARK-34701][SQL] Introduce AnalysisOnlyCommand that allows its children to be removed once the command is marked as analyzed
### What changes were proposed in this pull request?

This PR proposes to introduce the `AnalysisOnlyCommand` trait such that a command that extends this trait can have its children only analyzed, but not optimized. There is a corresponding analysis rule `HandleAnalysisOnlyCommand` that marks the command as analyzed after all other analysis rules are run.

This can be useful if a logical plan has children where they need to be only analyzed, but not optimized - e.g., `CREATE VIEW` or `CACHE TABLE AS`. This also addresses the issue found in #31933.

This PR also updates `CreateViewCommand`, `CacheTableAsSelect`, and `AlterViewAsCommand` to use the new trait / rule such that their children are only analyzed.

### Why are the changes needed?

To address the issue where the plan is unnecessarily re-analyzed in `CreateViewCommand`.

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

No.

### How was this patch tested?

Existing tests should cover the changes.

Closes #32032 from imback82/skip_transform.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 08:24:25 +00:00
Karen Feng 816f6dd13e [SPARK-34527][SQL] Resolve duplicated common columns from USING/NATURAL JOIN
### What changes were proposed in this pull request?

Adds the duplicated common columns as hidden columns to the Projection used to rewrite NATURAL/USING JOINs.

### Why are the changes needed?

Allows users to resolve either side of the NATURAL/USING JOIN's common keys.
Previously, the user could only resolve the following columns:

| Join type | Left key columns | Right key columns |
| --- | --- | --- |
| Inner | Yes | No |
| Left | Yes | No |
| Right | No | Yes |
| Outer | No | No |

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

Yes. The user can now symmetrically resolve the common columns from a NATURAL/USING JOIN.

### How was this patch tested?

SQL-side tests. The behavior matches PostgreSQL and MySQL.

Closes #31666 from karenfeng/spark-34527.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-14 07:01:40 +00:00
Kousuke Saruta ef05e89ee5 [SPARK-34977][SQL] LIST FILES/JARS/ARCHIVES cannot handle multiple arguments properly when at least one path is quoted
### What changes were proposed in this pull request?

This PR fixes an issue that `LIST FILES/JARS/ARCHIVES path1 path2 ...` cannot list all paths if at least one path is quoted.
An example here.
```
ADD FILE /tmp/test1;
ADD FILE /tmp/test2;

LIST FILES /tmp/test1 /tmp/test2;
file:/tmp/test1
file:/tmp/test2

LIST FILES /tmp/test1 "/tmp/test2";
file:/tmp/test2
```

In this example, the second `LIST FILES` doesn't show `file:/tmp/test1`.

To resolve this issue, I modified the syntax rule to be able to handle this case.
I also changed `SparkSQLParser` to be able to handle paths which contains white spaces.

### Why are the changes needed?

This is a bug.
I also have a plan which extends `ADD FILE/JAR/ARCHIVE` to take multiple paths like Hive and the syntax rule change is necessary for that.

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

Yes. Users can pass quoted paths when using `ADD FILE/JAR/ARCHIVE`.

### How was this patch tested?

New test.

Closes #32074 from sarutak/fix-list-files-bug.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-14 10:33:45 +09:00
gengjiaan 27bec91bc9 [SPARK-33604][SQL] Group exception messages in sql/execution
### What changes were proposed in this pull request?
This PR group exception messages in `/core/src/main/scala/org/apache/spark/sql/execution`.

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

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

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

Closes #31920 from beliefer/SPARK-33604.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 14:03:36 +00:00
Chao Sun 1a6708918b [SPARK-34947][SQL] Streaming write to a V2 table should invalidate its associated cache
### What changes were proposed in this pull request?

Populate table catalog and identifier from `DataStreamWriter` to `WriteToMicroBatchDataSource` so that we can invalidate cache for tables that are updated by a streaming write.

This is somewhat related [SPARK-27484](https://issues.apache.org/jira/browse/SPARK-27484) and [SPARK-34183](https://issues.apache.org/jira/browse/SPARK-34183) (#31700), as ideally we may want to replace `WriteToMicroBatchDataSource` and `WriteToDataSourceV2` with logical write nodes and feed them to analyzer. That will potentially change the code path involved in this PR.

### Why are the changes needed?

Currently `WriteToDataSourceV2` doesn't have cache invalidation logic, and therefore, when the target table for a micro batch streaming job is cached, the cache entry won't be removed when the table is updated.

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

Yes now when a DSv2 table which supports streaming write is updated by a streaming job, its cache will also be invalidated.

### How was this patch tested?

Added a new UT.

Closes #32039 from sunchao/streaming-cache.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 13:31:09 +00:00
Yuming Wang b34a84e21e [SPARK-34212][SQL][FOLLOWUP] Move the added test to ParquetQuerySuite
### What changes were proposed in this pull request?

This pr moves the added test from `SQLQuerySuite` to `ParquetQuerySuite`.

### Why are the changes needed?
1. It can be tested by `ParquetV1QuerySuite` and `ParquetV2QuerySuite`.
2. Reduce the testing time of `SQLQuerySuite`(SQLQuerySuite ~ 3 min 17 sec, ParquetV1QuerySuite ~ 27 sec).

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

No.

### How was this patch tested?

Unit test.

Closes #32090 from wangyum/SPARK-34212.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 09:04:47 +00:00
allisonwang-db 6b8405b574 [SPARK-28379][SQL] Allow non-aggregated single row correlated scalar subquery
### What changes were proposed in this pull request?
This PR allows non-aggregated correlated scalar subquery if the max output row is less than 2. Correlated scalar subqueries need to be aggregated because they are going to be decorrelated and rewritten as LEFT OUTER joins. If the correlated scalar subquery produces more than one output row, the rewrite will yield wrong results.

But this constraint can be relaxed when the subquery plan's the max number of output rows is less than or equal to 1.

### Why are the changes needed?
To relax a constraint in CheckAnalysis for the correlated scalar subquery.

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

### How was this patch tested?
Unit tests

Closes #32111 from allisonwang-db/spark-28379-aggregated.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-13 07:27:17 +00:00
ayushi agarwal caf33be274 [SPARK-33411][SQL] Cardinality estimation of union, sort and range operator
### What changes were proposed in this pull request?
Supports cardinality estimation of union, sort and range operator.

1. **Union**: number of rows in output will be the sum of number of rows in the output for each child of union, min and max for each column in the output will be the min and max of that particular column coming from its children.
Example:
Table 1
a   b
1   6
2   3
Table 2
a   b
1   3
 4   1
stats for table1 union table2 would be number of rows = 4, columnStats = (a: {min: 1, max: 4}, b: {min: 1, max: 6})

2. **Sort**: row and columns stats would be same as its children.

3. **Range**: number of output rows and distinct count will be equal to number of elements, min and max is calculated from start, end and step param.

### Why are the changes needed?
The change will enhance the feature https://issues.apache.org/jira/browse/SPARK-16026 and will help in other stats based optimizations.

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

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

Closes #30334 from ayushi-agarwal/SPARK-33411.

Lead-authored-by: ayushi agarwal <ayaga@microsoft.com>
Co-authored-by: ayushi-agarwal <36420535+ayushi-agarwal@users.noreply.github.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-13 15:17:17 +09:00
Max Gekk 26f312e95f [SPARK-35037][SQL] Recognize sign before the interval string in literals
### What changes were proposed in this pull request?
1. Extend SQL syntax rules to support a sign before the interval strings of ANSI year-month and day-time intervals.
2. Recognize `-` in `AstBuilder` and negate parsed intervals.

### Why are the changes needed?
To conform to the SQL standard which allows a sign before the string interval, see `"5.3 <literal>"`:
```
<interval literal> ::=
  INTERVAL [ <sign> ] <interval string> <interval qualifier>
<interval string> ::=
  <quote> <unquoted interval string> <quote>
<unquoted interval string> ::=
  [ <sign> ] { <year-month literal> | <day-time literal> }
<sign> ::=
    <plus sign>
  | <minus sign>
```

### Does this PR introduce _any_ user-facing change?
Should not because it just extends supported intervals syntax.

### How was this patch tested?
By running new tests in `interval.sql`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32134 from MaxGekk/negative-parsed-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-13 08:55:00 +03:00
Kent Yao 16e2faadac [SPARK-34944][SQL][TESTS] Replace bigint with int for web_returns and store_returns in TPCDS tests to employ correct data type
### What changes were proposed in this pull request?

According to  http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf

```
 2.2.2 Datatype
2.2.2.1 Each column employs one of the following datatypes:
a) Identifier means that the column shall be able to hold any key value generated for that column.
b) Integer means that the column shall be able to exactly represent integer values (i.e., values in increments of
1) in the range of at least ( − 2n − 1) to (2n − 1 − 1), where n is 64.
c) Decimal(d, f) means that the column shall be able to represent decimal values up to and including d digits,
of which f shall occur to the right of the decimal place; the values can be either represented exactly or
interpreted to be in this range.
d) Char(N) means that the column shall be able to hold any string of characters of a fixed length of N.
Comment: If the string that a column of datatype char(N) holds is shorter than N characters, then trailing
spaces shall be stored in the database or the database shall automatically pad with spaces upon retrieval such
that a CHAR_LENGTH() function will return N.
e) Varchar(N) means that the column shall be able to hold any string of characters of a variable length with a
maximum length of N. Columns defined as "varchar(N)" may optionally be implemented as "char(N)".
f) Date means that the column shall be able to express any calendar day between January 1, 1900 and
December 31, 2199.
2.2.2.2 The datatypes do not correspond to any specific SQL-standard datatype. The definitions are provided to
highlight the properties that are required for a particular column. The benchmark implementer may employ any internal representation or SQL datatype that meets those requirements.
```

This PR proposes that we use int for identifiers instead of bigint to reach a compromise with TPC-DS Standard Specification.

After this PR, the field schemas are now consistent with those DDLs in the `tpcds.sql` from tpc-ds tool kit, see https://gist.github.com/yaooqinn/b9978a77bbf4f871a95d6a9103019907

### Why are the changes needed?

reach a compromise with TPC-DS Standard Specification

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

no test only

### How was this patch tested?

test only

Closes #32037 from yaooqinn/SPARK-34944.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-04-13 11:28:35 +08:00
Gengliang Wang 79e55b44f7 [SPARK-35028][SQL] ANSI mode: disallow group by aliases
### What changes were proposed in this pull request?

Disallow group by aliases under ANSI mode.

### Why are the changes needed?

As per the ANSI SQL standard secion 7.12 <group by clause>:

>Each `grouping column reference` shall unambiguously reference a column of the table resulting from the `from clause`. A column referenced in a `group by clause` is a grouping column.

By forbidding it, we can avoid ambiguous SQL queries like:
```
SELECT col + 1 as col FROM t GROUP BY col
```

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

Yes, group by aliases is not allowed under ANSI mode.

### How was this patch tested?

Unit tests

Closes #32129 from gengliangwang/disallowGroupByAlias.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-13 10:42:57 +08:00
angerszhu 278203d969 [SPARK-28227][SQL] Support projection, aggregate/window functions, and lateral view in the TRANSFORM clause
### What changes were proposed in this pull request?
For Spark SQL, it can't support script transform SQL with aggregationClause/windowClause/LateralView.
This case we can't directly migration Hive SQL to Spark SQL.

In this PR, we treat all script transform statement's query part (exclude transform about part)  as a  separate query block and solve it as ScriptTransformation's child and pass a UnresolvedStart as ScriptTransform's input. Then in analyzer level, we pass child's output as ScriptTransform's input. Then we can support all kind of normal SELECT query combine with script transformation.

Such as transform with aggregation:
```
SELECT TRANSFORM ( d2, max(d1) as max_d1, sum(d3))
USING 'cat' AS (a,b,c)
FROM script_trans
WHERE d1 <= 100
GROUP BY d2
 HAVING max_d1 > 0
```
When we build AST, we treat it as
```
SELECT TRANSFORM (*)
USING 'cat' AS (a,b,c)
FROM (
     SELECT  d2, max(d1) as max_d1, sum(d3)
     FROM script_trans
    WHERE d1 <= 100
    GROUP BY d2
    HAVING max_d1 > 0
) tmp
```
then in Analyzer's `ResolveReferences`, resolve `* (UnresolvedStar)`, then sql behavior like
```
SELECT TRANSFORM ( d2, max(d1) as max_d1, sum(d3))
USING 'cat' AS (a,b,c)
FROM script_trans
WHERE d1 <= 100
GROUP BY d2
HAVING max_d1 > 0
```

About UT, in this pr we add a lot of different SQL to check we can support all kind of such SQL and  each kind of expressions can work well, such as alias, case when, binary compute etc...

### Why are the changes needed?
Support transform with aggregateClause/windowClause/LateralView etc , make sql migration more smoothly

### Does this PR introduce _any_ user-facing change?
User can write transform with  aggregateClause/windowClause/LateralView.

### How was this patch tested?
Added UT

Closes #29087 from AngersZhuuuu/SPARK-28227-NEW.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-13 11:34:45 +09:00
Wenchen Fan 8627cab39d [SPARK-34593][SQL][FOLLOWUP] Fix BroadcastNestedLoopJoinExec.outputPartition with full outer join
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31708 . For full outer join, the final result RDD is created from
```
sparkContext.union(
  matchedStreamRows,
  sparkContext.makeRDD(notMatchedBroadcastRows)
)
```

It's incorrect to say that the final output partitioning is `UnknownPartitioning(left.outputPartitioning.numPartitions)`

### Why are the changes needed?

Fix a correctness bug

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

Yes, see the added test. Fortunately, this bug is not released yet.

### How was this patch tested?

new test

Closes #32132 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-12 17:29:21 -07:00
Yuming Wang e40fce919a [SPARK-34562][SQL] Add test and doc for Parquet Bloom filter push down
### What changes were proposed in this pull request?

This pr add test and document for Parquet Bloom filter push down.

### Why are the changes needed?

Improve document.

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

No.

### How was this patch tested?

Generating docs:
![image](https://user-images.githubusercontent.com/5399861/114327472-c131bb80-9b6b-11eb-87a0-6f9a74eb1097.png)

Closes #32123 from wangyum/SPARK-34562.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-12 17:07:35 +03:00
Max Gekk 8f8bac6435 [SPARK-34905][SQL][TESTS] Enable ANSI intervals in SQLQueryTestSuite/ThriftServerQueryTestSuite
### What changes were proposed in this pull request?
Remove `spark.sql.legacy.interval.enabled` settings from `SQLQueryTestSuite`/`ThriftServerQueryTestSuite` that enables new ANSI intervals by default.

### Why are the changes needed?
To use default settings for intervals, and test new ANSI intervals - year-month and day-time interval introduced by SPARK-27793.

### Does this PR introduce _any_ user-facing change?
Should not because this affects tests only.

### How was this patch tested?
By running the affected tests, for instance:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32099 from MaxGekk/enable-ansi-intervals-sql-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-12 09:25:51 +00:00
Angerszhuuuu 21232377ba [SPARK-33229][SQL] Support partial grouping analytics and concatenated grouping analytics
### What changes were proposed in this pull request?
Support GROUP BY use Separate columns and CUBE/ROLLUP

In postgres sql, it support
```
select a, b, c, count(1) from t group by a, b, cube (a, b, c);
select a, b, c, count(1) from t group by a, b, rollup(a, b, c);
select a, b, c, count(1) from t group by cube(a, b), rollup (a, b, c);
select a, b, c, count(1) from t group by a, b, grouping sets((a, b), (a), ());
```
In this pr, we have done two things as below:

1. Support partial grouping analytics such as `group by a, cube(a, b)`
2. Support mixed grouping analytics such as `group by cube(a, b), rollup(b,c)`

*Partial Groupings*

    Partial Groupings means there are both `group_expression` and `CUBE|ROLLUP|GROUPING SETS`
    in GROUP BY clause. For example:
    `GROUP BY warehouse, CUBE(product, location)` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, product), (warehouse, location), (warehouse))`.
    `GROUP BY warehouse, ROLLUP(product, location)` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, product), (warehouse))`.
    `GROUP BY warehouse, GROUPING SETS((product, location), (producet), ())` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product, location), (warehouse, location), (warehouse))`.

*Concatenated Groupings*

    Concatenated groupings offer a concise way to generate useful combinations of groupings. Groupings specified
    with concatenated groupings yield the cross-product of groupings from each grouping set. The cross-product
    operation enables even a small number of concatenated groupings to generate a large number of final groups.
    The concatenated groupings are specified simply by listing multiple `GROUPING SETS`, `CUBES`, and `ROLLUP`,
    and separating them with commas. For example:
    `GROUP BY GROUPING SETS((warehouse), (producet)), GROUPING SETS((location), (size))` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, location), (warehouse, size), (product, location), (product, size))`.
    `GROUP BY CUBE((warehouse), (producet)), ROLLUP((location), (size))` is equivalent to
    `GROUP BY GROUPING SETS((warehouse, product), (warehouse), (producet), ()), GROUPING SETS((location, size), (location), ())`
    `GROUP BY GROUPING SETS(
        (warehouse, product, location, size), (warehouse, product, location), (warehouse, product),
        (warehouse, location, size), (warehouse, location), (warehouse),
        (product, location, size), (product, location), (product),
        (location, size), (location), ())`.
    `GROUP BY order, CUBE((warehouse), (producet)), ROLLUP((location), (size))` is equivalent to
    `GROUP BY order, GROUPING SETS((warehouse, product), (warehouse), (producet), ()), GROUPING SETS((location, size), (location), ())`
    `GROUP BY GROUPING SETS(
        (order, warehouse, product, location, size), (order, warehouse, product, location), (order, warehouse, product),
        (order, warehouse, location, size), (order, warehouse, location), (order, warehouse),
        (order, product, location, size), (order, product, location), (order, product),
        (order, location, size), (order, location), (order))`.

### Why are the changes needed?
Support more flexible grouping analytics

### Does this PR introduce _any_ user-facing change?
User can use sql like
```
select a, b, c, agg_expr() from table group by a, cube(b, c)
```

### How was this patch tested?
Added UT

Closes #30144 from AngersZhuuuu/SPARK-33229.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Wenchen Fan <cloud0fan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-12 08:23:52 +00:00
Angerszhuuuu 03431d40eb [SPARK-34986][SQL] Make an error msg clearer when ordinal numbers in group-by refer to agg funcs
### What changes were proposed in this pull request?
before when we use aggregate ordinal in group by expression and index position is a aggregate function, it will show error as
```
– !query
select a, b, sum(b) from data group by 3
– !query schema
struct<>
– !query output
org.apache.spark.sql.AnalysisException
aggregate functions are not allowed in GROUP BY, but found sum(data.b)
```

It't not clear enough refactor this error message in this pr

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

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

### How was this patch tested?
Existed UT

Closes #32089 from AngersZhuuuu/SPARK-34986.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-12 11:45:08 +09:00
Max Gekk 0e761c7307 [SPARK-35016][SQL] Format ANSI intervals in Hive style
### What changes were proposed in this pull request?
1. Extend `IntervalUtils` methods: `toYearMonthIntervalString` and `toDayTimeIntervalString` to support formatting of year-month/day-time intervals in Hive style. The methods get new parameter style which can have to values; `HIVE_STYLE` and `ANSI_STYLE`.
2. Invoke `toYearMonthIntervalString` and `toDayTimeIntervalString` from the `Cast` expression with the `style` parameter is set to `ANSI_STYLE`.
3. Invoke `toYearMonthIntervalString` and `toDayTimeIntervalString` from `HiveResult` with `style` is set to `HIVE_STYLE`.

### Why are the changes needed?
The `spark-sql` shell formats its output in Hive style by using `HiveResult.hiveResultString()`. The changes are needed to match Hive behavior. For instance,

Hive:
```sql
0: jdbc:hive2://localhost:10000/default> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
+-----------------------+
|          _c0          |
+-----------------------+
| 1 01:02:03.000001000  |
+-----------------------+
```

Spark before the changes:
```sql
spark-sql> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
INTERVAL '1 01:02:03.000001' DAY TO SECOND
```

Also this should unblock #32099 which enables *.sql tests in `SQLQueryTestSuite`.

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> select timestamp'2021-01-01 01:02:03.000001' - date'2020-12-31';
1 01:02:03.000001000
```

### How was this patch tested?
1. Added new tests to `IntervalUtilsSuite`:
```
$  build/sbt "test:testOnly *IntervalUtilsSuite"
```
2. Modified existing tests in `HiveResultSuite`:
```
$  build/sbt -Phive-2.3 -Phive-thriftserver "testOnly *HiveResultSuite"
```
3. By running cast tests:
```
$ build/sbt "testOnly *CastSuite*"
```

Closes #32120 from MaxGekk/ansi-intervals-hive-thrift-server.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-11 10:13:19 +03:00
Liang-Chi Hsieh 364d1eaf10 [SPARK-34963][SQL] Fix nested column pruning for extracting case-insensitive struct field from array of struct
### What changes were proposed in this pull request?

This patch proposes a fix of nested column pruning for extracting case-insensitive struct field from array of struct.

### Why are the changes needed?

Under case-insensitive mode, nested column pruning rule cannot correctly push down extractor of a struct field of an array of struct, e.g.,

```scala
val query = spark.table("contacts").select("friends.First", "friends.MiDDle")
```

Error stack:
```
[info]   java.lang.IllegalArgumentException: Field "First" does not exist.
[info] Available fields:
[info]   at org.apache.spark.sql.types.StructType$$anonfun$apply$1.apply(StructType.scala:274)
[info]   at org.apache.spark.sql.types.StructType$$anonfun$apply$1.apply(StructType.scala:274)
[info]   at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
[info]   at scala.collection.AbstractMap.getOrElse(Map.scala:59)
[info]   at org.apache.spark.sql.types.StructType.apply(StructType.scala:273)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema$$anonfun$getProjection$3.apply(ProjectionOverSchema.scala:44)
[info]   at org.apache.spark.sql.execution.ProjectionOverSchema$$anonfun$getProjection$3.apply(ProjectionOverSchema.scala:41)
```

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

No

### How was this patch tested?

Unit test

Closes #32059 from viirya/fix-array-nested-pruning.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-09 11:52:55 -07:00
Chao Sun ee7bf7d962 [SPARK-35003][SQL] Improve performance for reading smallint in vectorized Parquet reader
### What changes were proposed in this pull request?

Implements `readShorts` in `VectorizedPlainValuesReader`, which decodes `total` shorts in the input buffer at one time, similar to other types.

### Why are the changes needed?

Currently `VectorizedRleValuesReader` reads short integer in the following way:

```java
for (int i = 0; i < n; i++) {
  c.putShort(rowId + i, (short)data.readInteger());
}
```
For PLAIN encoding `data.readInteger` is done via:

```java
public final int readInteger() {
  return getBuffer(4).getInt();
}
```
which means it needs to repeatedly call `slice` buffer for the batch size number of times. This is more expensive than calling it once in a big chunk and then reading the ints out.

Micro benchmark via `DataSourceReadBenchmark` showed ~35% perf improvement.

Before:
```
[info] OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.16
[info] Intel(R) Core(TM) i9-9880H CPU  2.30GHz
[info] SQL Single SMALLINT Column Scan:          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] SQL CSV                                           10249          10271          32          1.5         651.6       1.0X
[info] SQL Json                                           5963           5982          28          2.6         379.1       1.7X
[info] SQL Parquet Vectorized                              141            151          15        111.9           8.9      72.9X
[info] SQL Parquet MR                                     1454           1491          52         10.8          92.4       7.0X
[info] SQL ORC Vectorized                                  160            164           3         98.3          10.2      64.1X
[info] SQL ORC MR                                         1133           1164          44         13.9          72.0       9.0X
```

After:
```
[info] OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.16
[info] Intel(R) Core(TM) i9-9880H CPU  2.30GHz
[info] SQL Single SMALLINT Column Scan:          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
[info] ------------------------------------------------------------------------------------------------------------------------
[info] SQL CSV                                           10489          10535          65          1.5         666.8       1.0X
[info] SQL Json                                           5864           5888          34          2.7         372.8       1.8X
[info] SQL Parquet Vectorized                              104            111           8        151.0           6.6     100.7X
[info] SQL Parquet MR                                     1458           1472          20         10.8          92.7       7.2X
[info] SQL ORC Vectorized                                  157            166           7        100.0          10.0      66.7X
[info] SQL ORC MR                                         1121           1147          37         14.0          71.2       9.4X
```

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

No

### How was this patch tested?

Existing tests

Closes #32104 from sunchao/smallint.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-09 08:12:47 -07:00
Ali Afroozeh 0945baf906 [SPARK-34989] Improve the performance of mapChildren and withNewChildren methods
### What changes were proposed in this pull request?
One of the main performance bottlenecks in query compilation is overly-generic tree transformation methods, namely `mapChildren` and `withNewChildren` (defined in `TreeNode`). These methods have an overly-generic implementation to iterate over the children and rely on reflection to create new instances. We have observed that, especially for queries with large query plans, a significant amount of CPU cycles are wasted in these methods. In this PR we make these methods more efficient, by delegating the iteration and instantiation to concrete node types. The benchmarks show that we can expect significant performance improvement in total query compilation time in queries with large query plans (from 30-80%) and about 20% on average.

#### Problem detail
The `mapChildren` method in `TreeNode` is overly generic and costly. To be more specific, this method:
- iterates over all the fields of a node using Scala’s product iterator. While the iteration is not reflection-based, thanks to the Scala compiler generating code for `Product`, we create many anonymous functions and visit many nested structures (recursive calls).
The anonymous functions (presumably compiled to Java anonymous inner classes) also show up quite high on the list in the object allocation profiles, so we are putting unnecessary pressure on GC here.
- does a lot of comparisons. Basically for each element returned from the product iterator, we check if it is a child (contained in the list of children) and then transform it. We can avoid that by just iterating over children, but in the current implementation, we need to gather all the fields (only transform the children) so that we can instantiate the object using the reflection.
- creates objects using reflection, by delegating to the `makeCopy` method, which is several orders of magnitude slower than using the constructor.

#### Solution
The proposed solution in this PR is rather straightforward: we rewrite the `mapChildren` method using the `children` and `withNewChildren` methods. The default `withNewChildren` method suffers from the same problems as `mapChildren` and we need to make it more efficient by specializing it in concrete classes.  Similar to how each concrete query plan node already defines its children, it should also define how they can be constructed given a new list of children. Actually, the implementation is quite simple in most cases and is a one-liner thanks to the copy method present in Scala case classes. Note that we cannot abstract over the copy method, it’s generated by the compiler for case classes if no other type higher in the hierarchy defines it. For most concrete nodes, the implementation of `withNewChildren` looks like this:
```
override def withNewChildren(newChildren: Seq[LogicalPlan]): LogicalPlan = copy(children = newChildren)
```
The current `withNewChildren` method has two properties that we should preserve:

- It returns the same instance if the provided children are the same as its children, i.e., it preserves referential equality.
- It copies tags and maintains the origin links when a new copy is created.

These properties are hard to enforce in the concrete node type implementation. Therefore, we propose a template method `withNewChildrenInternal` that should be rewritten by the concrete classes and let the `withNewChildren` method take care of referential equality and copying:
```
override def withNewChildren(newChildren: Seq[LogicalPlan]): LogicalPlan = {
 if (childrenFastEquals(children, newChildren)) {
   this
 } else {
   CurrentOrigin.withOrigin(origin) {
     val res = withNewChildrenInternal(newChildren)
     res.copyTagsFrom(this)
     res
   }
 }
}
```

With the refactoring done in a previous PR (https://github.com/apache/spark/pull/31932) most tree node types fall in one of the categories of `Leaf`, `Unary`, `Binary` or `Ternary`. These traits have a more efficient implementation for `mapChildren` and define a more specialized version of `withNewChildrenInternal` that avoids creating unnecessary lists. For example, the `mapChildren` method in `UnaryLike` is defined as follows:
```
  override final def mapChildren(f: T => T): T = {
    val newChild = f(child)
    if (newChild fastEquals child) {
      this.asInstanceOf[T]
    } else {
      CurrentOrigin.withOrigin(origin) {
        val res = withNewChildInternal(newChild)
        res.copyTagsFrom(this.asInstanceOf[T])
        res
      }
    }
  }
```

#### Results
With this PR, we have observed significant performance improvements in query compilation time, more specifically in the analysis and optimization phases. The table below shows the TPC-DS queries that had more than 25% speedup in compilation times. Biggest speedups are observed in queries with large query plans.
| Query  | Speedup |
| ------------- | ------------- |
|q4    |29%|
|q9    |81%|
|q14a  |31%|
|q14b  |28%|
|q22   |33%|
|q33   |29%|
|q34   |25%|
|q39   |27%|
|q41   |27%|
|q44   |26%|
|q47   |28%|
|q48   |76%|
|q49   |46%|
|q56   |26%|
|q58   |43%|
|q59   |46%|
|q60   |50%|
|q65   |59%|
|q66   |46%|
|q67   |52%|
|q69   |31%|
|q70   |30%|
|q96   |26%|
|q98   |32%|

#### Binary incompatibility
Changing the `withNewChildren` in `TreeNode` breaks the binary compatibility of the code compiled against older versions of Spark because now it is expected that concrete `TreeNode` subclasses all implement the `withNewChildrenInternal` method. This is a problem, for example, when users write custom expressions. This change is the right choice, since it forces all newly added expressions to Catalyst implement it in an efficient manner and will prevent future regressions.
Please note that we have not completely removed the old implementation and renamed it to `legacyWithNewChildren`. This method will be removed in the future and for now helps the transition. There are expressions such as `UpdateFields` that have a complex way of defining children. Writing `withNewChildren` for them requires refactoring the expression. For now, these expressions use the old, slow method. In a future PR we address these expressions.

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

This PR does not introduce user facing changes but my break binary compatibility of the code compiled against older versions. See the binary compatibility section.

### How was this patch tested?

This PR is mainly a refactoring and passes existing tests.

Closes #32030 from dbaliafroozeh/ImprovedMapChildren.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-09 15:06:26 +02:00
Gengliang Wang bfba7fadd2 [SPARK-34881][SQL][FOLLOWUP] Implement toString() and sql() methods for TRY_CAST
### What changes were proposed in this pull request?

Implement toString() and sql() methods for TRY_CAST

### Why are the changes needed?

The new expression should have a different name from `CAST` in SQL/String representation.

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

Yes, in the result of `explain()`, users can see try_cast if the new expression is used.

### How was this patch tested?

Unit tests.

Closes #32098 from gengliangwang/tryCastString.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-09 15:39:25 +08:00
Chao Sun 5013171fd3 [SPARK-34973][SQL] Cleanup unused fields and methods in vectorized Parquet reader
### What changes were proposed in this pull request?

Remove some unused fields and methods in `SpecificParquetRecordReaderBase` and `VectorizedColumnReader`.

### Why are the changes needed?

Some fields and methods in these classes are no longer used since years ago. It's better to clean them up to make the code easier to maintain and read.

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

No

### How was this patch tested?

Existing tests

Closes #32071 from sunchao/cleanup-parquet.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-08 11:21:07 -07:00
Max Gekk 96a3533de8 [SPARK-34984][SQL] ANSI intervals formatting in hive results
### What changes were proposed in this pull request?
Extend `HiveResult.toHiveString()` to support new interval types `YearMonthIntervalType` and `DayTimeIntervalType`.

### Why are the changes needed?
To fix failures while formatting ANSI intervals as Hive strings. For example:
```sql
spark-sql> select timestamp'now' - date'2021-01-01';
21/04/08 09:42:49 ERROR SparkSQLDriver: Failed in [select timestamp'now' - date'2021-01-01']
scala.MatchError: (PT2337H42M46.649S,DayTimeIntervalType) (of class scala.Tuple2)
	at org.apache.spark.sql.execution.HiveResult$.toHiveString(HiveResult.scala:97)
```

### Does this PR introduce _any_ user-facing change?
Yes. After the changes:
```sql
spark-sql> select timestamp'now' - date'2021-01-01';
INTERVAL '97 09:37:52.171' DAY TO SECOND
```

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

Closes #32087 from MaxGekk/ansi-interval-hiveResultString.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 16:18:15 +00:00
Tathagata Das c1c9a318c2 [SPARK-34962][SQL] Explicit representation of * in UpdateAction and InsertAction in MergeIntoTable
### What changes were proposed in this pull request?
Change UpdateAction and InsertAction of MergeIntoTable to explicitly represent star,

### Why are the changes needed?
Currently, UpdateAction and InsertAction in the MergeIntoTable implicitly represent `update set *` and `insert *` with empty assignments. That means there is no way to differentiate between the representations of "update all columns" and "update no columns". For SQL MERGE queries, this inability does not matter because the SQL MERGE grammar that generated the MergeIntoTable plan does not allow "update no columns". However, other ways of generating the MergeIntoTable plan may not have that limitation, and may want to allow specifying "update no columns". For example, in the Delta Lake project we provide a type-safe Scala API for Merge, where it is perfectly valid to produce a Merge query with an update clause but no update assignments. Currently, we cannot use MergeIntoTable to represent this plan, thus complicating the generation, and resolution of merge query from scala API.

Side note: fixed another bug where a merge plan with star and no other expressions with unresolved attributes (e.g. all non-optional predicates are `literal(true)`), then resolution will be skipped and star wont expanded. added test for that.

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

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

Closes #32067 from tdas/SPARK-34962-2.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 14:21:31 +00:00
Angerszhuuuu 90613df652 [SPARK-33233][SQL] CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal
### What changes were proposed in this pull request?
Currently, we can't support use ordinal in CUBE/ROLLUP/GROUPING SETS,
this pr make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal

### Why are the changes needed?
Make CUBE/ROLLUP/GROUPING SETS support GROUP BY ordinal.
Postgres SQL and TeraData support this use case.

### Does this PR introduce _any_ user-facing change?
User can use ordinal in CUBE/ROLLUP/GROUPING SETS, such as
```
-- can use ordinal in CUBE
select a, b, count(1) from data group by cube(1, 2);

-- mixed cases: can use ordinal in CUBE
select a, b, count(1) from data group by cube(1, b);

-- can use ordinal with cube
select a, b, count(1) from data group by 1, 2 with cube;

-- can use ordinal in ROLLUP
select a, b, count(1) from data group by rollup(1, 2);

-- mixed cases: can use ordinal in ROLLUP
select a, b, count(1) from data group by rollup(1, b);

-- can use ordinal with rollup
select a, b, count(1) from data group by 1, 2 with rollup;

-- can use ordinal in GROUPING SETS
select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));

-- mixed cases: can use ordinal in GROUPING SETS
select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));

select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));

```

### How was this patch tested?
Added UT

Closes #30145 from AngersZhuuuu/SPARK-33233.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-08 14:19:41 +00:00
Kousuke Saruta e5d972e84e [SPARK-34955][SQL] ADD JAR command cannot add jar files which contains whitespaces in the path
### What changes were proposed in this pull request?

This PR fixes an issue that `ADD JAR` command can't add jar files which contain whitespaces in the path though `ADD FILE` and `ADD ARCHIVE` work with such files.

If we have `/some/path/test file.jar` and execute the following command:

```
ADD JAR "/some/path/test file.jar";
```

The following exception is thrown.

```
21/04/05 10:40:38 ERROR SparkSQLDriver: Failed in [add jar "/some/path/test file.jar"]
java.lang.IllegalArgumentException: Illegal character in path at index 9: /some/path/test file.jar
	at java.net.URI.create(URI.java:852)
	at org.apache.spark.sql.hive.HiveSessionResourceLoader.addJar(HiveSessionStateBuilder.scala:129)
	at org.apache.spark.sql.execution.command.AddJarCommand.run(resources.scala:34)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
```

This is because `HiveSessionStateBuilder` and `SessionStateBuilder` don't check whether the form of the path is URI or plain path and it always regards the path as URI form.
Whitespces should be encoded to `%20` so `/some/path/test file.jar` is rejected.
We can resolve this part by checking whether the given path is URI form or not.

Unfortunatelly, if we fix this part, another problem occurs.
When we execute `ADD JAR` command, Hive's `ADD JAR` command is executed in `HiveClientImpl.addJar` and `AddResourceProcessor.run` is transitively invoked.
In `AddResourceProcessor.run`, the command line is just split by `
s+` and the path is also split into `/some/path/test` and `file.jar` and passed to `ss.add_resources`.
f1e8713703/ql/src/java/org/apache/hadoop/hive/ql/processors/AddResourceProcessor.java (L56-L75)
So, the command still fails.

Even if we convert the form of the path to URI like `file:/some/path/test%20file.jar` and execute the following command:

```
ADD JAR "file:/some/path/test%20file";
```

The following exception is thrown.

```
21/04/05 10:40:53 ERROR SessionState: file:/some/path/test%20file.jar does not exist
java.lang.IllegalArgumentException: file:/some/path/test%20file.jar does not exist
	at org.apache.hadoop.hive.ql.session.SessionState.validateFiles(SessionState.java:1168)
	at org.apache.hadoop.hive.ql.session.SessionState$ResourceType.preHook(SessionState.java:1289)
	at org.apache.hadoop.hive.ql.session.SessionState$ResourceType$1.preHook(SessionState.java:1278)
	at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1378)
	at org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1336)
	at org.apache.hadoop.hive.ql.processors.AddResourceProcessor.run(AddResourceProcessor.java:74)
```

The reason is `Utilities.realFile` invoked in `SessionState.validateFiles` returns `null` as the result of `fs.exists(path)` is `false`.
f1e8713703/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (L1052-L1064)

`fs.exists` checks the existence of the given path by comparing the string representation of Hadoop's `Path`.
The string representation of `Path` is similar to URI but it's actually different.
`Path` doesn't encode the given path.
For example, the URI form of `/some/path/jar file.jar` is `file:/some/path/jar%20file.jar` but the `Path` form of it is `file:/some/path/jar file.jar`. So `fs.exists` returns false.

So the solution I come up with is removing Hive's `ADD JAR` from `HiveClientimpl.addJar`.
I think Hive's `ADD JAR` was used to add jar files to the class loader for metadata and isolate the class loader from the one for execution.
https://github.com/apache/spark/pull/6758/files#diff-cdb07de713c84779a5308f65be47964af865e15f00eb9897ccf8a74908d581bbR94-R103

But, as of SPARK-10810 and SPARK-10902 (#8909) are resolved, the class loaders for metadata and execution seem to be isolated with different way.
https://github.com/apache/spark/pull/8909/files#diff-8ef7cabf145d3fe7081da799fa415189d9708892ed76d4d13dd20fa27021d149R635-R641

In the current implementation, such class loaders seem to be isolated by `SharedState.jarClassLoader` and `IsolatedClientLoader.classLoader`.

https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala#L173-L188
https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L956-L967

So I wonder we can remove Hive's `ADD JAR` from `HiveClientImpl.addJar`.
### Why are the changes needed?

This is a bug.

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

### How was this patch tested?

Closes #32052 from sarutak/add-jar-whitespace.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-07 11:43:03 -07:00
Gengliang Wang f208d80881 [SPARK-34970][SQL][SERCURITY] Redact map-type options in the output of explain()
### What changes were proposed in this pull request?

The `explain()` method prints the arguments of tree nodes in logical/physical plans. The arguments could contain a map-type option that contains sensitive data.
We should map-type options in the output of `explain()`. Otherwise, we will see sensitive data in explain output or Spark UI.
![image](https://user-images.githubusercontent.com/1097932/113719178-326ffb00-96a2-11eb-8a2c-28fca3e72941.png)

### Why are the changes needed?

Data security.

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

Yes, redact the map-type options in the output of `explain()`

### How was this patch tested?

Unit tests

Closes #32066 from gengliangwang/redactOptions.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-07 18:19:01 +08:00
Ali Afroozeh 06c09a79b3 [SPARK-34969][SPARK-34906][SQL] Followup for Refactor TreeNode's children handling methods into specialized traits
### What changes were proposed in this pull request?

This is a followup for https://github.com/apache/spark/pull/31932.
In this PR we:
- Introduce the `QuaternaryLike` trait for node types with 4 children.
- Specialize more node types
- Fix a number of style errors that were introduced in the original PR.

### Why are the changes needed?

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

### How was this patch tested?

This is a refactoring, passes existing tests.

Closes #32065 from dbaliafroozeh/FollowupSPARK-34906.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-04-07 09:50:30 +02:00
allisonwang-db 0aa2c284e4 [SPARK-34678][SQL] Add table function registry
### What changes were proposed in this pull request?
This PR extends the current function registry and catalog to support table-valued functions by adding a table function registry. It also refactors `range` to be a built-in function in the table function registry.

### Why are the changes needed?
Currently, Spark resolves table-valued functions very differently from the other functions. This change is to make the behavior for table and non-table functions consistent. It also allows Spark to display information about built-in table-valued functions:
Before:
```scala
scala> sql("describe function range").show(false)
+--------------------------+
|function_desc             |
+--------------------------+
|Function: range not found.|
+--------------------------+
```
After:
```scala
Function: range
Class: org.apache.spark.sql.catalyst.plans.logical.Range
Usage:
  range(start: Long, end: Long, step: Long, numPartitions: Int)
  range(start: Long, end: Long, step: Long)
  range(start: Long, end: Long)
  range(end: Long)

// Extended
Function: range
Class: org.apache.spark.sql.catalyst.plans.logical.Range
Usage:
  range(start: Long, end: Long, step: Long, numPartitions: Int)
  range(start: Long, end: Long, step: Long)
  range(start: Long, end: Long)
  range(end: Long)

Extended Usage:
  Examples:
    > SELECT * FROM range(1);
      +---+
      | id|
      +---+
      |  0|
      +---+
    > SELECT * FROM range(0, 2);
      +---+
      |id |
      +---+
      |0  |
      |1  |
      +---+
    > SELECT range(0, 4, 2);
      +---+
      |id |
      +---+
      |0  |
      |2  |
      +---+

    Since: 2.0.0
```

### Does this PR introduce _any_ user-facing change?
Yes. User will not be able to create a function with name `range` in the default database:
Before:
```scala
scala> sql("create function range as 'range'")
res3: org.apache.spark.sql.DataFrame = []
```
After:
```
scala> sql("create function range as 'range'")
org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException: Function 'default.range' already exists in database 'default'
```

### How was this patch tested?
Unit test

Closes #31791 from allisonwang-db/spark-34678-table-func-registry.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-07 05:49:36 +00:00
Tanel Kiis 7c8dc5e0b5 [SPARK-34922][SQL] Use a relative cost comparison function in the CBO
### What changes were proposed in this pull request?

Changed the cost comparison function of the CBO to use the ratios of row counts and sizes in bytes.

### Why are the changes needed?

In #30965 we changed to CBO cost comparison function so it would be "symetric": `A.betterThan(B)` now implies, that `!B.betterThan(A)`.
With that we caused a performance regressions in some queries - TPCDS q19 for example.

The original cost comparison function used the ratios `relativeRows = A.rowCount / B.rowCount` and `relativeSize = A.size / B.size`. The changed function compared "absolute" cost values `costA = w*A.rowCount + (1-w)*A.size` and `costB = w*B.rowCount + (1-w)*B.size`.

Given the input from wzhfy we decided to go back to the relative values, because otherwise one (size) may overwhelm the other (rowCount). But this time we avoid adding up the ratios.

Originally `A.betterThan(B) => w*relativeRows + (1-w)*relativeSize < 1` was used. Besides being "non-symteric", this also can exhibit one overwhelming other.
For `w=0.5` If `A` size (bytes) is at least 2x larger than `B`, then no matter how many times more rows does the `B` plan have, `B` will allways be considered to be better - `0.5*2 + 0.5*0.00000000000001 > 1`.

When working with ratios, then it would be better to multiply them.
The proposed cost comparison function is: `A.betterThan(B) => relativeRows^w  * relativeSize^(1-w) < 1`.

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

Comparison of the changed TPCDS v1.4 query execution times at sf=10:

  | absolute | multiplicative |   | additive |  
-- | -- | -- | -- | -- | --
q12 | 145 | 137 | -5.52% | 141 | -2.76%
q13 | 264 | 271 | 2.65% | 271 | 2.65%
q17 | 4521 | 4243 | -6.15% | 4348 | -3.83%
q18 | 758 | 466 | -38.52% | 480 | -36.68%
q19 | 38503 | 2167 | -94.37% | 2176 | -94.35%
q20 | 119 | 120 | 0.84% | 126 | 5.88%
q24a | 16429 | 16838 | 2.49% | 17103 | 4.10%
q24b | 16592 | 16999 | 2.45% | 17268 | 4.07%
q25 | 3558 | 3556 | -0.06% | 3675 | 3.29%
q33 | 362 | 361 | -0.28% | 380 | 4.97%
q52 | 1020 | 1032 | 1.18% | 1052 | 3.14%
q55 | 927 | 938 | 1.19% | 961 | 3.67%
q72 | 24169 | 13377 | -44.65% | 24306 | 0.57%
q81 | 1285 | 1185 | -7.78% | 1168 | -9.11%
q91 | 324 | 336 | 3.70% | 337 | 4.01%
q98 | 126 | 129 | 2.38% | 131 | 3.97%

All times are in ms, the change is compared to the situation in the master branch (absolute).
The proposed cost function (multiplicative) significantlly improves the performance on q18, q19 and q72. The original cost function (additive) has similar improvements at q18 and q19. All other chagnes are within the error bars and I would ignore them - perhaps q81 has also improved.

### How was this patch tested?

PlanStabilitySuite

Closes #32014 from tanelk/SPARK-34922_cbo_better_cost_function.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-07 11:31:10 +09:00
Wenchen Fan 19c7d2f3d8 Revert "[SPARK-34884][SQL] Improve DPP evaluation to make filtering side must can broadcast by size or broadcast by hint"
This reverts commit de66fa63f9.
2021-04-06 22:58:41 +08:00
Karen Feng 3b634f66c3 [SPARK-34923][SQL] Metadata output should be empty for more plans
### What changes were proposed in this pull request?

Changes the metadata propagation framework.

Previously, most `LogicalPlan`'s propagated their `children`'s `metadataOutput`. This did not make sense in cases where the `LogicalPlan` did not even propagate their `children`'s `output`.

I set the metadata output for plans that do not propagate their `children`'s `output` to be `Nil`. Notably, `Project` and `View` no longer have metadata output.

### Why are the changes needed?

Previously, `SELECT m from (SELECT a from tb)` would output `m` if it were metadata. This did not make sense.

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

Yes. Now, `SELECT m from (SELECT a from tb)` will encounter an `AnalysisException`.

### How was this patch tested?

Added unit tests. I did not cover all cases, as they are fairly extensive. However, the new tests cover major cases (and an existing test already covers Join).

Closes #32017 from karenfeng/spark-34923.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-06 16:04:30 +08:00
Kent Yao 7cffacef18 [SPARK-34935][SQL] CREATE TABLE LIKE should respect the reserved table properties
### What changes were proposed in this pull request?

CREATE TABLE LIKE should respect the reserved properties of tables and fail if specified, using `spark.sql.legacy.notReserveProperties` to restore.

### Why are the changes needed?

Make DDLs consistently treat reserved properties

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

YES, this is a breaking change as using `create table like` w/ reserved properties will fail.

### How was this patch tested?

new test

Closes #32025 from yaooqinn/SPARK-34935.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-06 08:52:48 +09:00
Dongjoon Hyun 748f05fca9 [SPARK-34954][SQL] Use zstd codec name in ORC file names
### What changes were proposed in this pull request?

This PR aims to add `zstd` codec names in the Spark generated ORC file names for consistency.

### Why are the changes needed?

Like the other ORC supported codecs, we had better have `zstd` in the Spark generated ORC file names. Please note that there is no problem at reading/writing ORC zstd files currently. This PR only aims to revise the file name format for consistency.

**SNAPPY**
```
scala> spark.range(10).repartition(1).write.option("compression", "snappy").orc("/tmp/snappy")

$ ls -al /tmp/snappy
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:17 .
drwxrwxrwt  14 root      wheel  448 Apr  4 12:17 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:17 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:17 .part-00000-833bb7ad-d1e1-48cc-9719-07b2d594aa4c-c000.snappy.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:17 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:17 part-00000-833bb7ad-d1e1-48cc-9719-07b2d594aa4c-c000.snappy.orc
```

**ZSTD (AS-IS)**
```
scala> spark.range(10).repartition(1).write.option("compression", "zstd").orc("/tmp/zstd")

$ ls -al /tmp/zstd
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:17 .
drwxrwxrwt  14 root      wheel  448 Apr  4 12:17 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:17 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:17 .part-00000-2f403ce9-7314-4db5-bca3-b1c1dd83335f-c000.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:17 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:17 part-00000-2f403ce9-7314-4db5-bca3-b1c1dd83335f-c000.orc
```

**ZSTD (After this PR)**
```
scala> spark.range(10).repartition(1).write.option("compression", "zstd").orc("/tmp/zstd_new")

$ ls -al /tmp/zstd_new
total 24
drwxr-xr-x   6 dongjoon  wheel  192 Apr  4 12:28 .
drwxrwxrwt  15 root      wheel  480 Apr  4 12:28 ..
-rw-r--r--   1 dongjoon  wheel    8 Apr  4 12:28 ._SUCCESS.crc
-rw-r--r--   1 dongjoon  wheel   12 Apr  4 12:28 .part-00000-49d57329-7196-4caf-839c-4251c876e26b-c000.zstd.orc.crc
-rw-r--r--   1 dongjoon  wheel    0 Apr  4 12:28 _SUCCESS
-rw-r--r--   1 dongjoon  wheel  231 Apr  4 12:28 part-00000-49d57329-7196-4caf-839c-4251c876e26b-c000.zstd.orc
```

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

No.

### How was this patch tested?

Pass the CIs with the updated UT.

Closes #32051 from dongjoon-hyun/SPARK-34954.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-04 17:11:56 -07:00
HyukjinKwon ebf01ec3c1 [SPARK-34950][TESTS] Update benchmark results to the ones created by GitHub Actions machines
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/32015 added a way to run benchmarks much more easily in the same GitHub Actions build. This PR updates the benchmark results by using the way.

**NOTE** that looks like GitHub Actions use four types of CPU given my observations:

- Intel(R) Xeon(R) Platinum 8171M CPU  2.60GHz
- Intel(R) Xeon(R) CPU E5-2673 v4  2.30GHz
- Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
- Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz

Given my quick research, seems like they perform roughly similarly:

![Screen Shot 2021-04-03 at 9 31 23 PM](https://user-images.githubusercontent.com/6477701/113478478-f4b57b80-94c3-11eb-9047-f81ca8c59672.png)

I couldn't find enough information about Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz but the performance seems roughly similar given the numbers.

So shouldn't be a big deal especially given that this way is much easier, encourages contributors to run more and guarantee the same number of cores and same memory with the same softwares.

### Why are the changes needed?

To have a base line of the benchmarks accordingly.

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

No, dev-only.

### How was this patch tested?

It was generated from:

- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

Closes #32044 from HyukjinKwon/SPARK-34950.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-03 23:02:56 +03:00
HyukjinKwon 71effba5f2 [SPARK-34821][INFRA] Set up a workflow for developers to run benchmark in their fork
### What changes were proposed in this pull request?

This PR proposes to add a workflow that allows developers to run benchmarks and download the results files.  After this PR, developers can run benchmarks in GitHub Actions in their fork.

### Why are the changes needed?

1. Very easy to use.
2. We can use the (almost) same environment to run the benchmarks. Given my few experiments and observation, the CPU, cores, and memory are same.
3. Does not burden ASF's resource at GitHub Actions.

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

No, dev-only.

### How was this patch tested?

Manually tested in https://github.com/HyukjinKwon/spark/pull/31.

Entire benchmarks are being run as below:
- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

### How do developers use it in their fork?

1. **Go to Actions in your fork, and click "Run benchmarks"**

    ![Screen Shot 2021-03-31 at 10 15 13 PM](https://user-images.githubusercontent.com/6477701/113150018-99d71680-926e-11eb-8647-4ecf062c55f2.png)

2. **Run the benchmarks with JDK 8 or 11 with benchmark classes to run. Glob pattern is supported just like `testOnly` in SBT**

    ![Screen Shot 2021-04-02 at 8 35 02 PM](https://user-images.githubusercontent.com/6477701/113412599-ab95f680-93f3-11eb-9a15-c6ed54587b9d.png)

3. **After finishing the jobs, the benchmark results are available on the top in the underlying workflow:**

    ![Screen Shot 2021-03-31 at 10 17 21 PM](https://user-images.githubusercontent.com/6477701/113150332-ede1fb00-926e-11eb-9c0e-97d195070508.png)

4. **After downloading it, unzip and untar at Spark git root directory:**

    ```bash
    cd .../spark
    mv ~/Downloads/benchmark-results-8.zip .
    unzip benchmark-results-8.zip
    tar -xvf benchmark-results-8.tar
    ```

5. **Check the results:**

    ```bash
    git status
    ```

    ```
    ...
        modified:   core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
    ```

Closes #32015 from HyukjinKwon/SPARK-34821-pr.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-03 20:55:54 +09:00
Angerszhuuuu 65da9287bc [SPARK-34926][SQL] PartitioningUtils.getPathFragment() should respect partition value is null
### What changes were proposed in this pull request?

When we insert data into a partition table partition with empty DataFrame. We will call `PartitioningUtils.getPathFragment()`
then to update this partition's metadata too.
When we insert to a partition when partition value is `null`, it will throw exception like
```
[info]   java.lang.NullPointerException:
[info]   at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:51)
[info]   at scala.collection.immutable.StringOps.length(StringOps.scala:51)
[info]   at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:35)
[info]   at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
[info]   at scala.collection.immutable.StringOps.foreach(StringOps.scala:33)
[info]   at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.escapePathName(ExternalCatalogUtils.scala:69)
[info]   at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.getPartitionValueString(ExternalCatalogUtils.scala:126)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.$anonfun$getPathFragment$1(PartitioningUtils.scala:354)
[info]   at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
[info]   at scala.collection.Iterator.foreach(Iterator.scala:941)
[info]   at scala.collection.Iterator.foreach$(Iterator.scala:941)
[info]   at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
[info]   at scala.collection.IterableLike.foreach(IterableLike.scala:74)
[info]   at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
```
`PartitioningUtils.getPathFragment()`  should support `null` value too

### Why are the changes needed?
Fix bug

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

### How was this patch tested?
Added UT

Closes #32018 from AngersZhuuuu/SPARK-34926.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-02 10:26:14 +03:00
Cheng Su 280a2f359c [SPARK-34940][SQL][TEST] Fix test of BasicWriteTaskStatsTrackerSuite
### What changes were proposed in this pull request?

This is to fix the minor typo in unit test of BasicWriteTaskStatsTrackerSuite (https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala#L152 ), where it should be a new file name, e.g. `f-3-3`, because the unit test expects 3 files in statistics (https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala#L160 ).

### Why are the changes needed?

Fix minor bug.

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

No.

### How was this patch tested?

Changed unit test `"Three files, last one empty"` itself.

Closes #32034 from c21/tracker-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-02 15:51:22 +09:00
Max Gekk 1d084513b9 [SPARK-34938][SQL][TESTS] Benchmark only legacy interval in ExtractBenchmark
### What changes were proposed in this pull request?
In the PR, I propose to disable ANSI intervals as the result of dates/timestamp subtraction in `ExtractBenchmark` and benchmark only legacy intervals because `EXTRACT( .. FROM ..)` doesn't support ANSI intervals so far.

### Why are the changes needed?
This fixes the benchmark failure:
```
[info]   Running case: YEAR of interval
[error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'year((subtractdates(CAST(timestamp_seconds(id) AS DATE), DATE '0001-01-01') + subtracttimestamps(timestamp_seconds(id), TIMESTAMP '1000-01-01 01:02:03.123456')))' due to data type mismatch: argument 1 requires date type, however, '(subtractdates(CAST(timestamp_seconds(id) AS DATE), DATE '0001-01-01') + subtracttimestamps(timestamp_seconds(id), TIMESTAMP '1000-01-01 01:02:03.123456'))' is of day-time interval type.; line 1 pos 0;
[error] 'Project [extract(YEAR, (subtractdates(cast(timestamp_seconds(id#1456L) as date), 0001-01-01, false) + subtracttimestamps(timestamp_seconds(id#1456L), 1000-01-01 01:02:03.123456, false, Some(Europe/Moscow)))) AS YEAR#1458]
[error] +- Range (1262304000, 1272304000, step=1, splits=Some(1))
[error] 	at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
[error] 	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:194)
```

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

### How was this patch tested?
By running the `ExtractBenchmark` benchmark via:
```
$ build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.ExtractBenchmark"
```

Closes #32035 from MaxGekk/fix-ExtractBenchmark.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-02 15:45:32 +09:00
yi.wu f897cc2374 [SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join
### What changes were proposed in this pull request?

This PR introduces a new analysis rule `DeduplicateRelations`, which deduplicates any duplicate relations in a plan first and then deduplicates conflicting attributes(which resued the `dedupRight` of `ResolveReferences`).

### Why are the changes needed?

`CostBasedJoinReorder` could fail when applying on self-join, e.g.,

```scala
// test in JoinReorderSuite
test("join reorder with self-join") {
  val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
      .select(nameToAttr("t1.v-1-10"))
      .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5")))

    // this can fail
    Optimize.execute(plan.analyze)
}
```
Besides, with the new rule `DeduplicateRelations`, we'd be able to enable some optimizations, e.g., LeftSemiAnti pushdown, redundant project removal, as reflects in updated unit tests.

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

### How was this patch tested?

Added and updated unit tests.

Closes #32027 from Ngone51/join-reorder-3.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-02 06:22:57 +00:00
Cheng Su 1fc66f6870 [SPARK-34862][SQL] Support nested column in ORC vectorized reader
### What changes were proposed in this pull request?

This PR is to support nested column type in Spark ORC vectorized reader. Currently ORC vectorized reader [does not support nested column type (struct, array and map)](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala#L138). We implemented nested column vectorized reader for FB-ORC in our internal fork of Spark. We are seeing performance improvement compared to non-vectorized reader when reading nested columns. In addition, this can also help improve the non-nested column performance when reading non-nested and nested columns together in one query.

Before this PR:

* `OrcColumnVector` is the implementation class for Spark's `ColumnVector` to wrap Hive's/ORC's `ColumnVector` to read `AtomicType` data.

After this PR:

* `OrcColumnVector` is an abstract class to keep interface being shared between multiple implementation class of orc column vectors, namely `OrcAtomicColumnVector` (for `AtomicType`), `OrcArrayColumnVector` (for `ArrayType`), `OrcMapColumnVector` (for `MapType`), `OrcStructColumnVector` (for `StructType`). So the original logic to read `AtomicType` data is moved from `OrcColumnVector` to `OrcAtomicColumnVector`. The abstract class of `OrcColumnVector` is needed here because of supporting nested column (i.e. nested column vectors).
* A utility method `OrcColumnVectorUtils.toOrcColumnVector` is added to create Spark's `OrcColumnVector` from Hive's/ORC's `ColumnVector`.
* A new user-facing config `spark.sql.orc.enableNestedColumnVectorizedReader` is added to control enabling/disabling vectorized reader for nested columns. The default value is false (i.e. disabling by default). For certain tables having deep nested columns, vectorized reader might take too much memory for each sub-column vectors, compared to non-vectorized reader. So providing a config here to work around OOM for query reading wide and deep nested columns if any. We plan to enable it by default on 3.3. Leave it disable in 3.2 in case for any unknown bugs.

### Why are the changes needed?

Improve query performance when reading nested columns from ORC file format.
Tested with locally adding a small benchmark in `OrcReadBenchmark.scala`. Seeing more than 1x run time improvement.

```
Running benchmark: SQL Nested Column Scan
  Running case: Native ORC MR
  Stopped after 2 iterations, 37850 ms
  Running case: Native ORC Vectorized (Enabled Nested Column)
  Stopped after 2 iterations, 15892 ms
  Running case: Native ORC Vectorized (Disabled Nested Column)
  Stopped after 2 iterations, 37954 ms
  Running case: Hive built-in ORC
  Stopped after 2 iterations, 35118 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
SQL Nested Column Scan:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------
Native ORC MR                                           18706          18925         310          0.1       17839.6       1.0X
Native ORC Vectorized (Enabled Nested Column)            7625           7946         455          0.1        7271.6       2.5X
Native ORC Vectorized (Disabled Nested Column)          18415          18977         796          0.1       17561.5       1.0X
Hive built-in ORC                                       17469          17559         127          0.1       16660.1       1.1X
```

Benchmark:

```
nestedColumnScanBenchmark(1024 * 1024)
def nestedColumnScanBenchmark(values: Int): Unit = {
    val benchmark = new Benchmark(s"SQL Nested Column Scan", values, output = output)

    withTempPath { dir =>
      withTempTable("t1", "nativeOrcTable", "hiveOrcTable") {
        import spark.implicits._
        spark.range(values).map(_ => Random.nextLong).map { x =>
          val arrayOfStructColumn = (0 until 5).map(i => (x + i, s"$x" * 5))
          val mapOfStructColumn = Map(
            s"$x" -> (x * 0.1, (x, s"$x" * 100)),
            (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)),
            (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300)))
          (arrayOfStructColumn, mapOfStructColumn)
        }.toDF("col1", "col2")
          .createOrReplaceTempView("t1")

        prepareTable(dir, spark.sql(s"SELECT * FROM t1"))

        benchmark.addCase("Native ORC MR") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Native ORC Vectorized (Enabled Nested Column)") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
        }

        benchmark.addCase("Native ORC Vectorized (Disabled Nested Column)") { _ =>
          withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") {
            spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM nativeOrcTable").noop()
          }
        }

        benchmark.addCase("Hive built-in ORC") { _ =>
          spark.sql("SELECT SUM(SIZE(col1)), SUM(SIZE(col2)) FROM hiveOrcTable").noop()
        }

        benchmark.run()
      }
    }
  }
```

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

No.

### How was this patch tested?

Added one simple test in `OrcSourceSuite.scala` to verify correctness.
Definitely need more unit tests and add benchmark here, but I want to first collect feedback before crafting more tests.

Closes #31958 from c21/orc-vector.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-04-01 23:10:34 -07:00
Kent Yao 1b553da2a1 [SPARK-34908][SQL][TESTS] Add test cases for char and varchar with functions
### What changes were proposed in this pull request?

Using char and varchar with the string functions and some other expressions might be confusing and ambiguous. In this PR we add test cases for char and varchar with these operations to reveal these behavior and see if we can come up with a general pattern for them.

### Why are the changes needed?

test coverage

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

no

### How was this patch tested?

new tests

Closes #32010 from yaooqinn/SPARK-34908.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-01 16:33:30 +09:00
Max Gekk 5911faa0d4 [SPARK-34903][SQL] Return day-time interval from timestamps subtraction
### What changes were proposed in this pull request?
Modify the `SubtractTimestamps` expression to return values of `DayTimeIntervalType` when `spark.sql.legacy.interval.enabled` is set to `false` (which is the default).

### Why are the changes needed?
To conform to the ANSI SQL standard which requires ANSI intervals as the result of timestamps subtraction, see
<img width="656" alt="Screenshot 2021-03-29 at 19 09 34" src="https://user-images.githubusercontent.com/1580697/112866455-7e2f0d00-90c2-11eb-96e6-3feb7eea7e09.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateTimeUtilsSuite"
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```
and some tests from `SQLQueryTestSuite`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z timestamp.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #32016 from MaxGekk/subtract-timestamps-to-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-01 10:27:58 +03:00
ulysses-you 89ae83d19b [SPARK-34919][SQL] Change partitioning to SinglePartition if partition number is 1
### What changes were proposed in this pull request?

Change partitioning to `SinglePartition`.

### Why are the changes needed?

For node `Repartition` and `RepartitionByExpression`, if partition number is 1 we can use `SinglePartition` instead of other `Partitioning`.

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

No

### How was this patch tested?

Add test

Closes #32012 from ulysses-you/SPARK-34919.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-01 06:59:31 +00:00
HyukjinKwon cc451c16a3 Revert "[SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join"
This reverts commit f05b940749.
2021-04-01 12:48:29 +09:00
Tanel Kiis 90f2d4d9cf [SPARK-34882][SQL] Replace if with filter clause in RewriteDistinctAggregates
### What changes were proposed in this pull request?

Replaced the `agg(if (('gid = 1)) 'cat1 else null)` pattern in `RewriteDistinctAggregates` with `agg('cat1) FILTER (WHERE 'gid = 1)`

### Why are the changes needed?

For aggregate functions, that do not ignore NULL values (`First`, `Last` or `UDAF`s) the current approach can return wrong results.

In the added UT there are no nulls in the input `testData`. The query returned `Row(0, 1, 0, 51, 100)` before this PR.

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

Bugfix

### How was this patch tested?

UT

Closes #31983 from tanelk/SPARK-34882_distinct_agg_filter.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-04-01 07:42:53 +09:00
Gengliang Wang 3951e3371a [SPARK-34881][SQL] New SQL Function: TRY_CAST
### What changes were proposed in this pull request?

Add a new SQL function `try_cast`.
`try_cast` is identical to  `AnsiCast` (or `Cast` when `spark.sql.ansi.enabled` is true), except it returns NULL instead of raising an error.
This expression has one major difference from `cast` with `spark.sql.ansi.enabled` as true: when the source value can't be stored in the target integral(Byte/Short/Int/Long) type, `try_cast` returns null instead of returning the low order bytes of the source value.
Note that the result of `try_cast` is not affected by the configuration `spark.sql.ansi.enabled`.

This is learned from Google BigQuery and Snowflake:
https://docs.snowflake.com/en/sql-reference/functions/try_cast.html
https://cloud.google.com/bigquery/docs/reference/standard-sql/functions-and-operators#safe_casting

### Why are the changes needed?

This is an useful for the following scenarios:
1. When ANSI mode is on, users can choose `try_cast` an alternative way to run SQL without errors for certain operations.
2. When ANSI mode is off, users can use `try_cast` to get a more reasonable result for casting a value to an integral type: when an overflow error happens, `try_cast` returns null while `cast` returns the low order bytes of the source value.

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

Yes, adding a new function `try_cast`

### How was this patch tested?

Unit tests.

Closes #31982 from gengliangwang/tryCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-31 20:47:04 +08:00
yi.wu f05b940749 [SPARK-34354][SQL] Fix failure when apply CostBasedJoinReorder on self-join
### What changes were proposed in this pull request?

This PR introduces a new analysis rule `DeduplicateRelations`, which deduplicates any duplicate relations in a plan first and then deduplicates conflicting attributes(which resued the `dedupRight` of `ResolveReferences`).

### Why are the changes needed?

`CostBasedJoinReorder` could fail when applying on self-join, e.g.,

```scala
// test in JoinReorderSuite
test("join reorder with self-join") {
  val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
      .select(nameToAttr("t1.v-1-10"))
      .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5")))

    // this can fail
    Optimize.execute(plan.analyze)
}
```
Besides, with the new rule `DeduplicateRelations`, we'd be able to enable some optimizations, e.g., LeftSemiAnti pushdown, redundant project removal, as reflects in updated unit tests.

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

### How was this patch tested?

Added and updated unit tests.

Closes #31470 from Ngone51/join-reorder.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 14:28:35 +08:00
Angerszhuuuu eecc43cb52 [SPARK-34568][SQL] When SparkContext's conf not enable hive, we should respect enableHiveSupport() when build SparkSession too
### What changes were proposed in this pull request?
When SparkContext is initialed, if we want to start SparkSession, when we call
`SparkSession.builder.enableHiveSupport().getOrCreate()`, the SparkSession we created won't have hive support since
we have't reset existed SC's conf's `spark.sql.catalogImplementation`.
In this PR we use sharedState.conf to decide whether we should enable Hive Support.

### Why are the changes needed?
We should respect `enableHiveSupport`

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

### How was this patch tested?
Added UT

Closes #31680 from AngersZhuuuu/SPARK-34568.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-31 05:59:24 +00:00
Max Gekk 162f0560e6 [SPARK-34896][SQL] Return day-time interval from dates subtraction
### What changes were proposed in this pull request?
1. Add the SQL config `spark.sql.legacy.interval.enabled` which will control when Spark SQL should use `CalendarIntervalType` instead of ANSI intervals.
2. Modify the `SubtractDates` expression to return values of `DayTimeIntervalType` when `spark.sql.legacy.interval.enabled` is set to `false` (which is the default).

### Why are the changes needed?
To conform to the ANSI SQL standard which requires ANSI intervals as the result of dates subtraction, see
<img width="656" alt="Screenshot 2021-03-29 at 19 09 34" src="https://user-images.githubusercontent.com/1580697/112866455-7e2f0d00-90c2-11eb-96e6-3feb7eea7e09.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *DateExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```
and some tests from `SQLQueryTestSuite`:
```
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z date.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z datetime.sql"
$ build/sbt "sql/testOnly *SQLQueryTestSuite -- -z interval.sql"
```

Closes #31996 from MaxGekk/subtract-dates-to-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-31 08:20:13 +03:00
Takeshi Yamamuro 46f96e9ce1 [SPARK-34795][SQL][TESTS] Adds a new job in GitHub Actions to check the output of TPC-DS queries
### What changes were proposed in this pull request?

This PR proposes to add a new job in GitHub Actions to check the output of TPC-DS queries.

NOTE: I've checked that the new job took 17m 35s in the GitHub Actions env.

### Why are the changes needed?

There are some cases where we noticed runtime-realted bugs after merging commits (e.g. .SPARK-33822). Therefore, I think it is worth adding a new job in GitHub Actions to check query output of TPC-DS (sf=1).

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

No.

### How was this patch tested?

The new test added.

Closes #31886 from maropu/TPCDSQueryTestSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-31 09:48:19 +09:00
Ali Afroozeh bd0990e3e8 [SPARK-34906] Refactor TreeNode's children handling methods into specialized traits
### What changes were proposed in this pull request?
Spark query plan node hierarchy has specialized traits (or abstract classes) for handling nodes with fixed number of children, for example `UnaryExpression`, `UnaryNode` and `UnaryExec` for representing an expression, a logical plan and a physical plan with only one child, respectively. This PR refactors the `TreeNode` hierarchy by extracting the children handling functionality into the following traits. `UnaryExpression` and other similar classes now extend the corresponding new trait:
```
trait LeafLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  override final def children: Seq[T] = Nil
}

trait UnaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def child: T
  transient override final lazy val children: Seq[T] = child :: Nil
}

trait BinaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def left: T
  def right: T
  transient override final lazy val children: Seq[T] = left :: right :: Nil
}

trait TernaryLike[T <: TreeNode[T]] { self: TreeNode[T] =>
  def first: T
  def second: T
  def third: T
  transient override final lazy val children: Seq[T] = first :: second :: third :: Nil
}
```

This refactoring, which is part of a bigger effort to make tree transformations in Spark more efficient, has two benefits:
- It moves the children handling methods to a single place, instead of being spread in specific subclasses, which will help the future optimizations for tree traversals.
- It allows to mix in these traits with some concrete node types that could not extend the previous classes. For example, expressions with one child that extend `AggregateFunction` cannot extend `UnaryExpression` as `AggregateFunction` defines the `foldable` method final while `UnaryExpression` defines it as non final. With the new traits, we can directly extend the concrete class from `UnaryLike` in these cases. Classes with more specific child handling will make tree traversal methods faster.

In this PR we have also updated many concrete node types to extend these traits to benefit from more specific child handling.

### Why are the changes needed?

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

### How was this patch tested?

This is a refactoring, passes existing tests.

Closes #31932 from dbaliafroozeh/FactorOutChildHandlnigIntoSeparateTraits.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2021-03-30 20:43:18 +02:00
ulysses-you 24d39a5ee2 [SPARK-34899][SQL] Use origin plan if we can not coalesce shuffle partition
### What changes were proposed in this pull request?

Add check if `CoalesceShufflePartitions` really coalesce shuffle partition number.

### Why are the changes needed?

The `CoalesceShufflePartitions` can not coalesce such case if the total shuffle partitions size of mappers are big enough. Then it's confused to use `CustomShuffleReaderExec` which marked as `coalesced` but has no affect with partition number.

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

Probably yes, the plan changed.

### How was this patch tested?

Add test.

Closes #31994 from ulysses-you/SPARK-34899.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 13:50:19 +00:00
Yuming Wang de66fa63f9 [SPARK-34884][SQL] Improve DPP evaluation to make filtering side must can broadcast by size or broadcast by hint
### What changes were proposed in this pull request?

Improve dynamic partition pruning evaluation to make filtering side must can broadcast by size or broadcast by hint.

### Why are the changes needed?

1. Fast fail if filtering side can not broadcast by size or broadcast by hint.
2. We can safely disable `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly`.

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

No.

### How was this patch tested?

Existing unit test.

Closes #31984 from wangyum/SPARK-34884.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:34:46 +00:00
angerszhu a98dc60408 [SPARK-33308][SQL] Refactor current grouping analytics
### What changes were proposed in this pull request?
As discussed in
https://github.com/apache/spark/pull/30145#discussion_r514728642
https://github.com/apache/spark/pull/30145#discussion_r514734648

We need to rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.
In  postgres sql, it support
```
select a, b, c, count(1) from t group by cube (a, b, c);
select a, b, c, count(1) from t group by cube(a, b, c);
select a, b, c, count(1) from t group by cube (a, b, c, (a, b), (a, b, c));
select a, b, c, count(1) from t group by rollup(a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c);
select a, b, c, count(1) from t group by rollup (a, b, c, (a, b), (a, b, c));
```
In this pr,  we have done three things as below, and we will split it to different pr:

 - Refactor CUBE/ROLLUP (regarding them as ANTLR tokens in a parser)
 - Refactor GROUPING SETS (the logical node -> a new expr)
 - Support new syntax for CUBE/ROLLUP (e.g., GROUP BY CUBE ((a, b), (a, c)))

### Why are the changes needed?
Rewrite current Grouping Analytics grammar to support  as flexible as Postgres SQL to support subsequent development.

### Does this PR introduce _any_ user-facing change?
User can  write Grouping Analytics grammar as flexible as Postgres SQL to support subsequent development.

### How was this patch tested?
Added UT

Closes #30212 from AngersZhuuuu/refact-grouping-analytics.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-30 12:31:58 +00:00
Cheng Su 935aa8c8db [SPARK-32985][SQL][FOLLOWUP] Rename createNonBucketedReadRDD and minor change in FileSourceScanExec
### What changes were proposed in this pull request?

This PR is a followup change to address comments in https://github.com/apache/spark/pull/31413#discussion_r603280965 and https://github.com/apache/spark/pull/31413#discussion_r603296475 . Minor change in `FileSourceScanExec`. No actual logic change here.

### Why are the changes needed?

Better readability.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #32000 from c21/bucket-scan.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 19:57:32 +09:00
yangjie01 7158e7f986 [SPARK-34900][TEST] Make sure benchmarks can run using spark-submit cmd described in the guide
### What changes were proposed in this pull request?
Some `spark-submit`  commands used to run benchmarks in the user's guide is wrong, we can't use these commands to run benchmarks successful.

So the major changes of this pr is correct these wrong commands, for example, run a benchmark which inherits from `SqlBasedBenchmark`, we must specify `--jars <spark core test jar>,<spark catalyst test jar>` because `SqlBasedBenchmark` based benchmark extends `BenchmarkBase(defined in spark core test jar)` and `SQLHelper(defined in spark catalyst test jar)`.

Another change of this pr is removed the `scalatest Assertions` dependency of Benchmarks because `scalatest-*.jar` are not in the distribution package, it will be troublesome to use.

### Why are the changes needed?
Make sure benchmarks can run using spark-submit cmd described in the guide

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

### How was this patch tested?
Use the corrected `spark-submit` commands to run benchmarks successfully.

Closes #31995 from LuciferYang/fix-benchmark-guide.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 11:58:01 +09:00
Jungtaek Lim 43e08b1f0f [SPARK-34255][SQL] Support partitioning with static number on required distribution and ordering on V2 write
### What changes were proposed in this pull request?

This PR proposes to extend the functionality of requirement for distribution and ordering on V2 write to specify the number of partitioning on repartition, so that data source is able to control the parallelism and determine the data distribution per partition in prior.

The partitioning with static number is optional, and by default disabled via default method, so only implementations required to restrict the number of partition statically need to override the method and provide the number.

Note that we don't support static number of partitions with unspecified distribution for this PR, as we haven't found the real use cases, and for hypothetical case the static number isn't good enough. Javadoc clearly describes the limitation.

### Why are the changes needed?

The use case comes from feature parity with DSv1.

I have state data source which enables the state in SS to be rewritten, which enables repartitioning, schema evolution, etc via batch query. The writer requires hash partitioning against group key, with the "desired number of partitions", which is same as what Spark does read and write against state.

This is now implemented as DSv1, and the requirement is simply done by calling repartition with the "desired number".

```
val fullPathsForKeyColumns = keySchema.map(key => new Column(s"key.${key.name}"))
data
  .repartition(newPartitions, fullPathsForKeyColumns: _*)
  .queryExecution
  .toRdd
  .foreachPartition(
    writeFn(resolvedCpLocation, version, operatorId, storeName, keySchema, valueSchema,
      storeConf, hadoopConfBroadcast, queryId))
```

Thanks to SPARK-34026, it's now possible to require the hash partitioning, but still not able to require the number of partitions. This PR will enable to let data source require the number of partitions.

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

Yes, but only for data source implementors. Even for them, this is no breaking change as default method is added.

### How was this patch tested?

Added UTs.

Closes #31355 from HeartSaVioR/SPARK-34255.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 14:33:23 +00:00
Kousuke Saruta 14c7bb877d [SPARK-34872][SQL] quoteIfNeeded should quote a name which contains non-word characters
### What changes were proposed in this pull request?

This PR fixes an issue that `quoteIfNeeded` quotes a name only if it contains `.` or ``` ` ```.
This method should quote it if it contains non-word characters.

### Why are the changes needed?

It's a potential bug.

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

No.

### How was this patch tested?

New test.

Closes #31964 from sarutak/fix-quoteIfNeeded.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-29 09:31:24 +00:00
Tanel Kiis 4b9e94c444 [SPARK-34876][SQL] Fill defaultResult of non-nullable aggregates
### What changes were proposed in this pull request?

Filled the `defaultResult` field on non-nullable aggregates

### Why are the changes needed?

The `defaultResult` defaults to `None` and in some situations (like correlated scalar subqueries) it is used for the value of the aggregation.

The UT result before the fix:
```
-- !query
SELECT t1a,
   (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2,
   (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2,
   (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2,
   (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2,
   (SELECT collect_set(t2d) FROM t2 WHERE t2a = t1a) collect_set_t2,
    (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2
FROM t1
-- !query schema
struct<t1a:string,count_t2:bigint,count_if_t2:bigint,approx_count_distinct_t2:bigint,collect_list_t2:array<bigint>,collect_set_t2:array<bigint>,collect_set_t2:string>
-- !query output
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1a	0	0	NULL	NULL	NULL	NULL
val1b	6	6	3	[19,119,319,19,19,19]	[19,119,319]	0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001
val1c	2	2	2	[219,19]	[219,19]	0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1d	0	0	NULL	NULL	NULL	NULL
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
val1e	1	1	1	[19]	[19]	0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000
```

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

Bugfix

### How was this patch tested?

UT

Closes #31973 from tanelk/SPARK-34876_non_nullable_agg_subquery.

Authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-29 11:47:08 +09:00
hanover-fiste 4fceef0159 [SPARK-34843][SQL] Calculate more precise partition stride in JDBCRelation
### What changes were proposed in this pull request?
The changes being proposed are to increase the accuracy of JDBCRelation's stride calculation, as outlined in: https://issues.apache.org/jira/browse/SPARK-34843

In summary:

Currently, in JDBCRelation (line 123), the stride size is calculated as follows:
val stride: Long = upperBound / numPartitions - lowerBound / numPartitions

Due to truncation happening on both divisions, the stride size can fall short of what it should be. This can lead to a big difference between the provided upper bound and the actual start of the last partition.

I'm proposing a different formula that doesn't truncate to early, and also maintains accuracy using fixed-point decimals. This helps tremendously with the size of the last partition, which can be even more amplified if there is data skew in that direction. In a real-life test, I've seen a 27% increase in performance with this more proper stride alignment. The reason for fixed-point decimals instead of floating-point decimals is because inaccuracy due to limitation of what the float can represent. This may seem small, but could shift the midpoint a bit, and depending on how granular the data is, that could translate to quite a difference. It's also just inaccurate, and I'm striving to make the partitioning as accurate as possible, within reason.

Lastly, since the last partition's predicate is determined by how the strides align starting from the lower bound (plus one stride), there can be skew introduced creating a larger last partition compared to the first partition. Therefore, after calculating a more precise stride size, I've also introduced logic to move the first partition's predicate (which is an offset from the lower bound) to a position that closely matches the offset of the last partition's predicate (in relation to the upper bound). This makes the first and last partition more evenly distributed compared to each other, and helps with the last task being the largest (reducing its size).

### Why are the changes needed?
The current implementation is inaccurate and can lead to the last task/partition running much longer than previous tasks. Therefore, you can end up with a single node/core running for an extended period while other nodes/cores are sitting idle.

### Does this PR introduce _any_ user-facing change?
No. I would suspect some users will just get a good performance increase. As stated above, if we were to run our code on Spark that has this change implemented, we would have all of the sudden got a 27% increase in performance.

### How was this patch tested?
I've added two new unit tests. I did need to update one unit test, but when you look at the comparison of the before and after, you'll see better alignment of the partitioning with the new implementation. Given that the lower partition's predicate is exclusive and the upper's is inclusive, the offset of the lower was 3 days, and the offset of the upper was 6 days... that's potentially twice the amount of data in that upper partition (could be much more depending on how the user's data is distributed).

Other unit tests that utilize timestamps and two partitions have maintained their midpoint.

### Examples

I've added results with and without the realignment logic to better highlight both improvements this PR brings.

**Example 1:**
Given the following partition config:
"lowerBound" -> "1930-01-01"
"upperBound" -> "2020-12-31"
"numPartitions" -> 1000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1930-02-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-07-11'
_Old method, but with new realingment logic of first partition:_
First partition: "PartitionColumn" < '1931-10-14' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2019-03-22'

_New method:_
First partition: "PartitionColumn" < '1930-02-03' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-05'
_New with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1930-06-02' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-08-02'

**Example 2:**
Given the following partition config:
"lowerBound" -> "1927-04-05",
"upperBound" -> "2020-10-16"
"numPartitions" -> 2000

_Old method (exactly what it would be BEFORE this PR):_
First partition: "PartitionColumn" < '1927-04-21' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2014-10-29'
_Old method, but with new realingment logic of first partition::_
First partition: "PartitionColumn" < '1930-04-07' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2017-10-15'

_New method:_
First partition: "PartitionColumn" < '1927-04-22' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-04-19'
_New method with new realingment logic of first partition (exactly what it would be AFTER this PR):_
First partition: "PartitionColumn" < '1927-07-13' or "PartitionColumn" is null
Last partition: "PartitionColumn" >= '2020-07-10'

Closes #31965 from hanover-fiste/SPARK-34843.

Authored-by: hanover-fiste <jyarbrough.git@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-28 12:59:20 -05:00
Peter Toth 3382190349 [SPARK-34829][SQL] Fix higher order function results
### What changes were proposed in this pull request?
This PR fixes a correctness issue with higher order functions. The results of function expressions needs to be copied in some higher order functions as such an expression can return with internal buffers and higher order functions can call multiple times the expression.
The issue was discovered with typed `ScalaUDF`s after https://github.com/apache/spark/pull/28979.

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

### Does this PR introduce _any_ user-facing change?
Yes, some queries return the right results again.

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

Closes #31955 from peter-toth/SPARK-34829-fix-scalaudf-resultconversion.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 10:01:09 -07:00
Yuming Wang 540f1fb1d9 [SPARK-32855][SQL][FOLLOWUP] Fix code format in SQLConf and comment in PartitionPruning
### What changes were proposed in this pull request?

Fix code format in `SQLConf` and comment in `PartitionPruning`.

### Why are the changes needed?

Make code more readable.

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

No.

### How was this patch tested?

N/A

Closes #31969 from wangyum/SPARK-32855-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 09:48:54 -07:00
Dongjoon Hyun e7af44861e [SPARK-34880][SQL][TESTS] Add Parquet ZSTD compression test coverage
### What changes were proposed in this pull request?

Apache Parquet 1.12.0 switches its ZSTD compression from Hadoop codec to its own codec.

### Why are the changes needed?

**Apache Spark 3.1 (It requires libhadoop built with zstd)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
21/03/27 08:49:38 ERROR Executor: Exception in task 11.0 in stage 0.0 (TID 11)2]
java.lang.RuntimeException: native zStandard library not available:
this version of libhadoop was built without zstd support.
```

**Apache Spark 3.2 (No libhadoop requirement)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
```

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

Yes, this is an improvement.

### How was this patch tested?

Pass the CI with the newly added test coverage.

Closes #31981 from dongjoon-hyun/SPARK-34880.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 12:48:12 -07:00
Angerszhuuuu 769cf7b966 [SPARK-34744][SQL] Improve error message for casting cause overflow error
### What changes were proposed in this pull request?
Improve error message for casting cause overflow error. We should use DataType's catalogString.

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

### Does this PR introduce _any_ user-facing change?
For example:
```
set spark.sql.ansi.enabled=true;
select tinyint(128) * tinyint(2);
```
Error message before this pr:
```
Casting 128 to scala.Byte$ causes overflow
```
After this pr:
```
Casting 128 to tinyint causes overflow
```

### How was this patch tested?
Added UT

Closes #31971 from AngersZhuuuu/SPARK-34744.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-27 11:15:55 +08:00
Max Gekk 9ba889b6ea [SPARK-34875][SQL] Support divide a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `day-time interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31972 from MaxGekk/div-dt-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 15:36:08 +00:00
Wenchen Fan 61d038f26e Revert "[SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand"
This reverts commit da04f1f4f8.
2021-03-26 15:26:48 +08:00
Max Gekk f212c61c43 [SPARK-34868][SQL] Support divide an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `year-month interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31961 from MaxGekk/div-ym-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 05:56:56 +00:00
Yuming Wang aaa0d2a66b [SPARK-32855][SQL] Improve the cost model in pruningHasBenefit for filtering side can not build broadcast by join type
### What changes were proposed in this pull request?

This pr improve the cost model in `pruningHasBenefit` for filtering side can not build broadcast by join type:
1. The filtering side must be small enough to build broadcast by size.
2. The estimated size of the pruning side must be big enough: `estimatePruningSideSize * spark.sql.optimizer.dynamicPartitionPruning.pruningSideExtraFilterRatio > overhead`.

### Why are the changes needed?

Improve query performance for these cases.

This a real case from cluster. Left join and left size very small and right side can build DPP:
![image](https://user-images.githubusercontent.com/5399861/92882197-445a2a00-f442-11ea-955d-16a7724e535b.png)

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

No.

### How was this patch tested?

Unit test.

Closes #29726 from wangyum/SPARK-32855.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 04:48:13 +00:00