Commit graph

10945 commits

Author SHA1 Message Date
Yuming Wang 540f1fb1d9 [SPARK-32855][SQL][FOLLOWUP] Fix code format in SQLConf and comment in PartitionPruning
### What changes were proposed in this pull request?

Fix code format in `SQLConf` and comment in `PartitionPruning`.

### Why are the changes needed?

Make code more readable.

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

No.

### How was this patch tested?

N/A

Closes #31969 from wangyum/SPARK-32855-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-28 09:48:54 -07:00
Dongjoon Hyun e7af44861e [SPARK-34880][SQL][TESTS] Add Parquet ZSTD compression test coverage
### What changes were proposed in this pull request?

Apache Parquet 1.12.0 switches its ZSTD compression from Hadoop codec to its own codec.

### Why are the changes needed?

**Apache Spark 3.1 (It requires libhadoop built with zstd)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
21/03/27 08:49:38 ERROR Executor: Exception in task 11.0 in stage 0.0 (TID 11)2]
java.lang.RuntimeException: native zStandard library not available:
this version of libhadoop was built without zstd support.
```

**Apache Spark 3.2 (No libhadoop requirement)**
```scala
scala> spark.range(10).write.option("compression", "zstd").parquet("/tmp/a")
```

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

Yes, this is an improvement.

### How was this patch tested?

Pass the CI with the newly added test coverage.

Closes #31981 from dongjoon-hyun/SPARK-34880.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 12:48:12 -07:00
Yuming Wang cbffc12f90 [SPARK-34542][BUILD] Upgrade Parquet to 1.12.0
### What changes were proposed in this pull request?

Parquet 1.12.0 New Feature
- PARQUET-41 - Add bloom filters to parquet statistics
- PARQUET-1373 - Encryption key management tools
- PARQUET-1396 - Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory
- PARQUET-1622 - Add BYTE_STREAM_SPLIT encoding
- PARQUET-1784 - Column-wise configuration
- PARQUET-1817 - Crypto Properties Factory
- PARQUET-1854 - Properties-Driven Interface to Parquet Encryption

Parquet 1.12.0 release notes:
https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/CHANGES.md

### Why are the changes needed?

- Bloom filters to improve filter performance
- ZSTD enhancement

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

No.

### How was this patch tested?

Existing unit test.

Closes #31649 from wangyum/SPARK-34542.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <yumwang@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 07:56:29 -07:00
Angerszhuuuu 468b944b00 [SPARK-34841][SQL] Push ANSI interval binary expressions into into (if/else) branches
### What changes were proposed in this pull request?
Push ANSI interval binary expressions into into (if / case) branches

### Why are the changes needed?
Support more binary expression to push into if/else and casewhen

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

### How was this patch tested?
Added UT

Closes #31978 from AngersZhuuuu/SPARK-34841.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-27 14:50:28 +03:00
Angerszhuuuu 769cf7b966 [SPARK-34744][SQL] Improve error message for casting cause overflow error
### What changes were proposed in this pull request?
Improve error message for casting cause overflow error. We should use DataType's catalogString.

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

### Does this PR introduce _any_ user-facing change?
For example:
```
set spark.sql.ansi.enabled=true;
select tinyint(128) * tinyint(2);
```
Error message before this pr:
```
Casting 128 to scala.Byte$ causes overflow
```
After this pr:
```
Casting 128 to tinyint causes overflow
```

### How was this patch tested?
Added UT

Closes #31971 from AngersZhuuuu/SPARK-34744.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-27 11:15:55 +08:00
Max Gekk 9ba889b6ea [SPARK-34875][SQL] Support divide a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `day-time interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31972 from MaxGekk/div-dt-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 15:36:08 +00:00
Wenchen Fan 61d038f26e Revert "[SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand"
This reverts commit da04f1f4f8.
2021-03-26 15:26:48 +08:00
Max Gekk f212c61c43 [SPARK-34868][SQL] Support divide an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `DivideYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `year-month interval / numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="656" alt="Screenshot 2021-03-25 at 18 44 58" src="https://user-images.githubusercontent.com/1580697/112501559-68f07080-8d9a-11eb-8781-66e6631bb7ef.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31961 from MaxGekk/div-ym-interval-by-num.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 05:56:56 +00:00
Yuming Wang aaa0d2a66b [SPARK-32855][SQL] Improve the cost model in pruningHasBenefit for filtering side can not build broadcast by join type
### What changes were proposed in this pull request?

This pr improve the cost model in `pruningHasBenefit` for filtering side can not build broadcast by join type:
1. The filtering side must be small enough to build broadcast by size.
2. The estimated size of the pruning side must be big enough: `estimatePruningSideSize * spark.sql.optimizer.dynamicPartitionPruning.pruningSideExtraFilterRatio > overhead`.

### Why are the changes needed?

Improve query performance for these cases.

This a real case from cluster. Left join and left size very small and right side can build DPP:
![image](https://user-images.githubusercontent.com/5399861/92882197-445a2a00-f442-11ea-955d-16a7724e535b.png)

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

No.

### How was this patch tested?

Unit test.

Closes #29726 from wangyum/SPARK-32855.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-26 04:48:13 +00:00
Kent Yao 820b465886 [SPARK-34786][SQL] Read Parquet unsigned int64 logical type that stored as signed int64 physical type to decimal(20, 0)
### What changes were proposed in this pull request?

A companion PR for SPARK-34817, when we handle the unsigned int(<=32) logical types. In this PR, we map the unsigned int64 to decimal(20, 0) for better compatibility.

### Why are the changes needed?

Spark won't have unsigned types, but spark should be able to read existing parquet files written by other systems that support unsigned types for better compatibility.

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

yes, we can read parquet uint64 now

### How was this patch tested?

new unit tests

Closes #31960 from yaooqinn/SPARK-34786-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-26 09:54:19 +08:00
Yuanjian Li 5ffc3897e0 [SPARK-34871][SS] Move the checkpoint location resolving into the rule ResolveWriteToStream
### What changes were proposed in this pull request?
Move the checkpoint location resolving into the rule ResolveWriteToStream, which is added in SPARK-34748.

### Why are the changes needed?
After SPARK-34748, we have a rule ResolveWriteToStream for the analysis logic for the resolving logic of stream write plans. Based on it, we can further move the checkpoint location resolving work in the rule. Then, all the checkpoint resolving logic was done in the analyzer.

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

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

Closes #31963 from xuanyuanking/SPARK-34871.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-26 10:29:50 +09:00
Wenchen Fan 658e95c345 [SPARK-34833][SQL][FOLLOWUP] Handle outer references in all the places
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/31940 . This PR generalizes the matching of attributes and outer references, so that outer references are handled everywhere.

Note that, currently correlated subquery has a lot of limitations in Spark, and the newly covered cases are not possible to happen. So this PR is a code refactor.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

Closes #31959 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-26 09:10:03 +09:00
Gengliang Wang 0515f49018 [SPARK-34856][SQL] ANSI mode: Allow casting complex types as string type
### What changes were proposed in this pull request?

Allow casting complex types as string type in ANSI mode.

### Why are the changes needed?

Currently, complex types are not allowed to cast as string type. This breaks the DataFrame.show() API. E.g
```
scala> sql(“select array(1, 2, 2)“).show(false)
org.apache.spark.sql.AnalysisException: cannot resolve ‘CAST(`array(1, 2, 2)` AS STRING)’ due to data type mismatch:
 cannot cast array<int> to string with ANSI mode on.
```
We should allow the conversion as the extension of the ANSI SQL standard, so that the DataFrame.show() still work in ANSI mode.
### Does this PR introduce _any_ user-facing change?

Yes, casting complex types as string type is now allowed in ANSI mode.

### How was this patch tested?

Unit tests.

Closes #31954 from gengliangwang/fixExplicitCast.

Authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-26 00:17:43 +08:00
Karen Feng 0d91f9c3f3 [SPARK-33600][SQL] Group exception messages in execution/datasources/v2
### What changes were proposed in this pull request?

This PR groups exception messages in `execution/datasources/v2`.

### Why are the changes needed?

It will largely help with standardization of error messages and its maintenance.

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

No. Error messages remain unchanged.

### How was this patch tested?

No new tests - pass all original tests to make sure it doesn't break any existing behavior.

Closes #31619 from karenfeng/spark-33600.

Authored-by: Karen Feng <karen.feng@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 16:15:30 +00:00
Tanel Kiis 6ba8445ea3 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

This is resubmition of #31927. The schema for one of the TPCDS tables was updated and that changed the `explain.txt` for the q17.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31957 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 10:22:49 +00:00
Tim Armstrong 1d6acd584a [SPARK-34857][SQL] Correct AtLeastNNonNulls's explain output
### What changes were proposed in this pull request?
Removed the custom toString implementation of AtLeastNNoneNulls.

### Why are the changes needed?
It shows up wrong in the explain plan. The name of the function is wrong and the actual value of the first argument is not shown. Both of these would make it easier to understand the plan.

```
(12) Filter
Input [3]: [c1#2410L, c2#2419, c3#2422]
Condition : AtLeastNNulls(n, c1#2410L)
```

### Does this PR introduce _any_ user-facing change?
Only the explain plan changes if this function is used.

### How was this patch tested?
Added a simple unit test to make sure that the toString output is correct.

Closes #31956 from timarmstrong/atleastnnonnulls.

Authored-by: Tim Armstrong <tim.armstrong@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-25 17:20:01 +09:00
Max Gekk a68d7ca8c5 [SPARK-34850][SQL] Support multiply a day-time interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyDTInterval` which multiplies a `DayTimeIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * day-time interval` and `day-time interval * numeric`.
3. Invoke `DoubleMath.roundToInt` in `double/float * year-month interval`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over day-time intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31951 from MaxGekk/mul-day-time-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-25 10:46:50 +03:00
Kent Yao 8c6748f691 [SPARK-34817][SQL] Read parquet unsigned types that stored as int32 physical type in parquet
### What changes were proposed in this pull request?

Unsigned types may be used to produce smaller in-memory representations of the data. These types used by frameworks(e.g. hive, pig) using parquet. And parquet will map them to its base types.

see more https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift

```thrift
  /**
   * An unsigned integer value.
   *
   * The number describes the maximum number of meaningful data bits in
   * the stored value. 8, 16 and 32 bit values are stored using the
   * INT32 physical type.  64 bit values are stored using the INT64
   * physical type.
   *
   */
  UINT_8 = 11;
  UINT_16 = 12;
  UINT_32 = 13;
  UINT_64 = 14;
```

```
UInt8-[0:255]
UInt16-[0:65535]
UInt32-[0:4294967295]
UInt64-[0:18446744073709551615]
```

In this PR, we support read UINT_8 as ShortType, UINT_16 as IntegerType, UINT_32 as LongType to fit their range. Support for UINT_64 will be in another PR.

### Why are the changes needed?

better parquet support

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

yes, we can read unit[8/16/32] from parquet files

### How was this patch tested?

new tests

Closes #31921 from yaooqinn/SPARK-34817.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:58:06 +00:00
Terry Kim da04f1f4f8 [SPARK-34701][SQL] Remove analyzing temp view again in CreateViewCommand
### What changes were proposed in this pull request?

This PR proposes to remove re-analyzing the already analyzed plan for `CreateViewCommand` as discussed https://github.com/apache/spark/pull/31273/files#r581592786.

### Why are the changes needed?

No need to analyze the plan if it's already analyzed.

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

No.

### How was this patch tested?

Existing tests should cover this.

Closes #31933 from imback82/remove_analyzed_from_create_temp_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-25 06:53:59 +00:00
HyukjinKwon 7838f55ca7 Revert "[SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes"
This reverts commit 84df54b495.
2021-03-25 12:31:08 +09:00
ulysses-you 9d561e6b5e [SPARK-34852][SQL] Close Hive session state should use withHiveState
### What changes were proposed in this pull request?

Wrap Hive sessionStae `close` with `withHiveState`

### Why are the changes needed?

Some reason:

1. Shutdown hook is invoked using different thread
2. Hive may use metasotre client again during closing

Otherwise, we may get such expcetion with custom hive metastore version
```
21/03/24 13:26:18 INFO session.SessionState: Failed to remove classloaders from DataNucleus
java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1654)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:80)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:130)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:101)
	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3367)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3406)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3386)
	at org.apache.hadoop.hive.ql.session.SessionState.unCacheDataNucleusClassLoaders(SessionState.java:1546)
	at org.apache.hadoop.hive.ql.session.SessionState.close(SessionState.java:1536)
	at org.apache.spark.sql.hive.client.HiveClientImpl.closeState(HiveClientImpl.scala:172)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$new$1(HiveClientImpl.scala:175)
	at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)
	at org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
```

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

No, since this not released.

### How was this patch tested?

manual test.

Closes #31949 from ulysses-you/SPARK-34852.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
2021-03-25 10:21:44 +08:00
Takeshi Yamamuro 150769bced [SPARK-34833][SQL] Apply right-padding correctly for correlated subqueries
### What changes were proposed in this pull request?

This PR intends to fix the bug that does not apply right-padding for char types inside correlated subquries.
For example,  a query below returns nothing in master, but a correct result is `c`.
```
scala> sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING parquet")
scala> sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(7)) USING parquet")
scala> sql("INSERT INTO t1 VALUES ('c', 'b')")
scala> sql("INSERT INTO t2 VALUES ('a', 'b')")
scala> val df = sql("""
  |SELECT v FROM t1
  |WHERE 'a' IN (SELECT v FROM t2 WHERE t2.c = t1.c )""".stripMargin)

scala> df.show()
+---+
|  v|
+---+
+---+

```

This is because `ApplyCharTypePadding`  does not handle the case above to apply right-padding into `'abc'`. This PR modifies the code in `ApplyCharTypePadding` for handling it correctly.

```
// Before this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#13]
+- Filter a IN (list#12 [c#14])
   :  +- Project [v#15]
   :     +- Filter (c#16 = outer(c#14))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#15,c#16] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#13,c#14] parquet

scala> df.show()
+---+
|  v|
+---+
+---+

// After this PR:
scala> df.explain(true)
== Analyzed Logical Plan ==
v: string
Project [v#43]
+- Filter a IN (list#42 [c#44])
   :  +- Project [v#45]
   :     +- Filter (c#46 = rpad(outer(c#44), 7,  ))
   :        +- SubqueryAlias spark_catalog.default.t2
   :           +- Relation default.t2[v#45,c#46] parquet
   +- SubqueryAlias spark_catalog.default.t1
      +- Relation default.t1[v#43,c#44] parquet

scala> df.show()
+---+
|  v|
+---+
|  c|
+---+
```

This fix is lated to TPCDS q17; the query returns nothing because of this bug: https://github.com/apache/spark/pull/31886/files#r599333799

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Unit tests added.

Closes #31940 from maropu/FixCharPadding.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-25 08:31:57 +09:00
Gengliang Wang abfd9b23cd [SPARK-34769][SQL] AnsiTypeCoercion: return closest convertible type among TypeCollection
### What changes were proposed in this pull request?

Currently, when implicit casting a data type to a `TypeCollection`, Spark returns the first convertible data type among `TypeCollection`.
In ANSI mode, we can make the behavior more reasonable by returning the closet convertible data type in `TypeCollection`.

In details, we first try to find the all the expected types we can implicitly cast:
1. if there is no convertible data types, return None;
2. if there is only one convertible data type, cast input as it;
3. otherwise if there are multiple convertible data types, find the closet data
type among them. If there is no such closet data type, return None.

Note that if the closet type is Float type and the convertible types contains Double type, simply return Double type as the closet type to avoid potential
precision loss on converting the Integral type as Float type.

### Why are the changes needed?

Make the type coercion rule for TypeCollection more reasonable and ANSI compatible.
E.g. returning Long instead of Double for`implicast(int, TypeCollect(Double, Long))`.

From ANSI SQL Spec section 4.33 "SQL-invoked routines"
![Screen Shot 2021-03-17 at 4 05 06 PM](https://user-images.githubusercontent.com/1097932/111434916-5e104e80-86bd-11eb-8b3b-33090a68067d.png)

Section 9.6 "Subject routine determination"
![Screen Shot 2021-03-17 at 1 36 55 PM](https://user-images.githubusercontent.com/1097932/111420336-48445e80-86a8-11eb-9d50-34b325043bdb.png)

Section 10.4 "routine invocation"
![Screen Shot 2021-03-17 at 4 08 41 PM](https://user-images.githubusercontent.com/1097932/111434926-610b3f00-86bd-11eb-8c32-8c7935e055da.png)

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

Yes, in ANSI mode, implicit casting to a `TypeCollection` returns the narrowest convertible data type instead of the first convertible one.

### How was this patch tested?

Unit tests.

Closes #31859 from gengliangwang/implicitCastTypeCollection.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 15:04:03 +00:00
Tanel Kiis 84df54b495 [SPARK-34822][SQL] Update the plan stability golden files even if only the explain.txt changes
### What changes were proposed in this pull request?

Update the plan stability golden files even if only the `explain.txt` changes.

### Why are the changes needed?

Currently only `simplified.txt` change is checked. There are some PRs, that update the `explain.txt`, that do not change the `simplified.txt`.

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

No

### How was this patch tested?

The updated golden files.

Closes #31927 from tanelk/SPARK-34822_update_plan_stability.

Lead-authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Co-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 14:36:51 +00:00
Cheng Su 35c70e417d [SPARK-34853][SQL] Remove duplicated definition of output partitioning/ordering for limit operator
### What changes were proposed in this pull request?

Both local limit and global limit define the output partitioning and output ordering in the same way and this is duplicated (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L159-L175 ). We can move the output partitioning and ordering into their parent trait - `BaseLimitExec`. This is doable as `BaseLimitExec` has no more other child class. This is a minor code refactoring.

### Why are the changes needed?

Clean up the code a little bit. Better readability.

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

No.

### How was this patch tested?

Pure refactoring. Rely on existing unit tests.

Closes #31950 from c21/limit-cleanup.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-24 23:06:35 +09:00
yangjie01 712a62ca82 [SPARK-34832][SQL][TEST] Set EXECUTOR_ALLOW_SPARK_CONTEXT to true to ensure ExternalAppendOnlyUnsafeRowArrayBenchmark run successfully
### What changes were proposed in this pull request?
SPARK-32160 add a config(`EXECUTOR_ALLOW_SPARK_CONTEXT`) to switch allow/disallow to create `SparkContext` in executors and the default value of the config is `false`

`ExternalAppendOnlyUnsafeRowArrayBenchmark` will run fail when `EXECUTOR_ALLOW_SPARK_CONTEXT` use the default value because the `ExternalAppendOnlyUnsafeRowArrayBenchmark#withFakeTaskContext` method try to create a `SparkContext` manually in Executor Side.

So the main change of this pr is  set `EXECUTOR_ALLOW_SPARK_CONTEXT` to `true` to ensure `ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

### Why are the changes needed?
Bug fix.

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

### How was this patch tested?
Manual test:
```
bin/spark-submit --class org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark --jars spark-core_2.12-3.2.0-SNAPSHOT-tests.jar spark-sql_2.12-3.2.0-SNAPSHOT-tests.jar
```

**Before**
```
Exception in thread "main" java.lang.IllegalStateException: SparkContext should only be created and accessed on the driver.
	at org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$assertOnDriver(SparkContext.scala:2679)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:89)
	at org.apache.spark.SparkContext.<init>(SparkContext.scala:137)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.withFakeTaskContext(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:52)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.testAgainstRawArrayBuffer(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:119)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.$anonfun$runBenchmarkSuite$1(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:189)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.benchmark.BenchmarkBase.runBenchmark(BenchmarkBase.scala:40)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark$.runBenchmarkSuite(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala:186)
	at org.apache.spark.benchmark.BenchmarkBase.main(BenchmarkBase.scala:58)
	at org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark.main(ExternalAppendOnlyUnsafeRowArrayBenchmark.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

**After**

`ExternalAppendOnlyUnsafeRowArrayBenchmark` run successfully.

Closes #31939 from LuciferYang/SPARK-34832.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-24 14:59:31 +09:00
Kousuke Saruta f7e9b6efc7 [SPARK-34763][SQL] col(), $"<name>" and df("name") should handle quoted column names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `col()`, `$"<name>"` and `df("name")` don't handle quoted column names  like ``` `a``b.c` ```properly.

For example, if we have a following DataFrame.
```
val df1 = spark.sql("SELECT 'col1' AS `a``b.c`")
```

For the DataFrame, this query is successfully executed.
```
scala> df1.selectExpr("`a``b.c`").show
+-----+
|a`b.c|
+-----+
| col1|
+-----+
```

But the following query will fail because ``` df1("`a``b.c`") ``` throws an exception.
```
scala> df1.select(df1("`a``b.c`")).show
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `a``b.c`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:152)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:162)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121)
  at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221)
  at org.apache.spark.sql.Dataset.col(Dataset.scala:1274)
  at org.apache.spark.sql.Dataset.apply(Dataset.scala:1241)
  ... 49 elided
```
### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

New tests.

Closes #31854 from sarutak/fix-parseAttributeName.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-24 13:34:10 +08:00
Takeshi Yamamuro 0494dc90af [SPARK-34842][SQL][TESTS] Corrects the type of date_dim.d_quarter_name in the TPCDS schema
### What changes were proposed in this pull request?

SPARK-34842 (#31012) has a typo in the type of `date_dim.d_quarter_name` in the TPCDS schema (`TPCDSBase`). This PR replace `CHAR(1)` with `CHAR(6)`. This fix comes from p28 in [the TPCDS official doc](http://www.tpc.org/tpc_documents_current_versions/pdf/tpc-ds_v2.9.0.pdf).

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

N/A

Closes #31943 from maropu/SPARK-34083-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-23 10:22:13 -07:00
Max Gekk 760556a42f [SPARK-34824][SQL] Support multiply an year-month interval by a numeric
### What changes were proposed in this pull request?
1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`.

### Why are the changes needed?
To conform the ANSI SQL standard which requires such operation over year-month intervals:
<img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">

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

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *IntervalExpressionsSuite"
$ build/sbt "test:testOnly *ColumnExpressionSuite"
```

Closes #31929 from MaxGekk/interval-mul-div.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-23 19:40:15 +03:00
Wenchen Fan 3b70829b5b [SPARK-34719][SQL] Correctly resolve the view query with duplicated column names
forward-port https://github.com/apache/spark/pull/31811 to master

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

For permanent views (and the new SQL temp view in Spark 3.1), we store the view SQL text and re-parse/analyze the view SQL text when reading the view. In the case of `SELECT * FROM ...`, we want to avoid view schema change (e.g. the referenced table changes its schema) and will record the view query output column names when creating the view, so that when reading the view we can add a `SELECT recorded_column_names FROM ...` to retain the original view query schema.

In Spark 3.1 and before, the final SELECT is added after the analysis phase: https://github.com/apache/spark/blob/branch-3.1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala#L67

If the view query has duplicated output column names, we always pick the first column when reading a view. A simple repro:
```
scala> sql("create view c(x, y) as select 1 a, 2 a")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("select * from c").show
+---+---+
|  x|  y|
+---+---+
|  1|  1|
+---+---+
```

In the master branch, we will fail at the view reading time due to b891862fb6 , which adds the final SELECT during analysis, so that the query fails with `Reference 'a' is ambiguous`

This PR proposes to resolve the view query output column names from the matching attributes by ordinal.

For example,  `create view c(x, y) as select 1 a, 2 a`, the view query output column names are `[a, a]`. When we reading the view, there are 2 matching attributes (e.g.`[a#1, a#2]`) and we can simply match them by ordinal.

A negative example is
```
create table t(a int)
create view v as select *, 1 as col from t
replace table t(a int, col int)
```
When reading the view, the view query output column names are `[a, col]`, and there are two matching attributes of `col`, and we should fail the query. See the tests for details.

### Why are the changes needed?

bug fix

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

yes

### How was this patch tested?

new test

Closes #31930 from cloud-fan/view2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 14:34:51 +00:00
Liang-Chi Hsieh 115ed89a3c [SPARK-34366][SQL] Add interface for DS v2 metrics
### What changes were proposed in this pull request?

This patch proposes to add a few public API change to DS v2, to make DS v2 scan can report metrics to Spark.

Two public interfaces are added.

* `CustomMetric`: metric interface at the driver side. It basically defines how Spark aggregates task metrics with the same metric name.
* `CustomTaskMetric`: task metric reported at executors. It includes a name and long value. Spark will collect these metric values and update internal metrics.

There are two public methods added to existing public interfaces. They are optional to DS v2 implementations.

* `PartitionReader.currentMetricsValues()`: returns an array of CustomTaskMetric. Here is where the actual metrics values are collected. Empty array by default.
* `Scan.supportedCustomMetrics()`: returns an array of supported custom metrics `CustomMetric`. Empty array by default.

### Why are the changes needed?

In order to report custom metrics, we need some public API change in DS v2 to make it possible.

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

No

### How was this patch tested?

This only adds interfaces. In follow-up PRs where adding implementation there will be tests added. See #31451 and #31398 for some details and manual test there.

Closes #31476 from viirya/SPARK-34366.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 13:22:37 +00:00
Peter Toth 93a5d34f84 [SPARK-33482][SPARK-34756][SQL] Fix FileScan equality check
### What changes were proposed in this pull request?

This bug was introduced by SPARK-30428 at Apache Spark 3.0.0.
This PR fixes `FileScan.equals()`.

### Why are the changes needed?
- Without this fix `FileScan.equals` doesn't take `fileIndex` and `readSchema` into account.
- Partition filters and data filters added to `FileScan` (in #27112 and #27157) caused that canonicalized form of some `BatchScanExec` nodes don't match and this prevents some reuse possibilities.

### Does this PR introduce _any_ user-facing change?
Yes, before this fix incorrect reuse of `FileScan` and so `BatchScanExec` could have happed causing correctness issues.

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

Closes #31848 from peter-toth/SPARK-34756-fix-filescan-equality-check.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 17:01:16 +08:00
yi.wu e00afd31a7 [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself
### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

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

No.

### How was this patch tested?

Pass existing tests.

Closes #31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:38:43 +00:00
linzebing e768eaa908 [SPARK-34707][SQL] Code-gen broadcast nested loop join (left outer/right outer)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left outer (build right) and right outer (build left). Reference: `BroadcastNestedLoopJoinExec.codegenInner()` and `BroadcastNestedLoopJoinExec.outerJoin()`

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:
```scala
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left outer broadcast nested loop join", N) {
   val df = spark.range(N).selectExpr(s"id as k1").join(
     dim, col("k1") + 1 <= col("k2"), "left_outer")
   assert(df.queryExecution.sparkPlan.find(
     _.isInstanceOf[BroadcastNestedLoopJoinExec]).isDefined)
   df.noop()
}
```
Seeing 2x run time improvement:
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left outer broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------------------
left outer broadcast nested loop join wholestage off           3024           3698         953          6.9         144.2       1.0X
left outer broadcast nested loop join wholestage on            1512           1659         172         13.9          72.1       2.0X
```

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

No

### How was this patch tested?

Changed existing unit tests in `OuterJoinSuite` to cover codegen use cases.
Added unit test in WholeStageCodegenSuite.scala to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:
```scala
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_outer").explain("codegen")
```
Example generated code (`bnlj_doConsume_0` method):
```java
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:210(0.32% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, LeftOuter, ((k1#2L + 1) <= k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=16)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=16)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 031 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_foundMatch_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */       boolean bnlj_shouldOutputRow_0 = false;
/* 041 */
/* 042 */       boolean bnlj_isNull_2 = true;
/* 043 */       long bnlj_value_2 = -1L;
/* 044 */       if (bnlj_buildRow_0 != null) {
/* 045 */         long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 046 */         bnlj_isNull_2 = false;
/* 047 */         bnlj_value_2 = bnlj_value_1;
/* 048 */       }
/* 049 */
/* 050 */       long bnlj_value_4 = -1L;
/* 051 */
/* 052 */       bnlj_value_4 = bnlj_expr_0_0 + 1L;
/* 053 */
/* 054 */       boolean bnlj_value_3 = false;
/* 055 */       bnlj_value_3 = bnlj_value_4 <= bnlj_value_2;
/* 056 */       if (!(false || !bnlj_value_3))
/* 057 */       {
/* 058 */         bnlj_shouldOutputRow_0 = true;
/* 059 */         bnlj_foundMatch_0 = true;
/* 060 */       }
/* 061 */       if (bnlj_arrayIndex_0 == bnlj_buildRowArray_0.length - 1 && !bnlj_foundMatch_0) {
/* 062 */         bnlj_buildRow_0 = null;
/* 063 */         bnlj_shouldOutputRow_0 = true;
/* 064 */       }
/* 065 */       if (bnlj_shouldOutputRow_0) {
/* 066 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 067 */
/* 068 */         boolean bnlj_isNull_9 = true;
/* 069 */         long bnlj_value_9 = -1L;
/* 070 */         if (bnlj_buildRow_0 != null) {
/* 071 */           long bnlj_value_8 = bnlj_buildRow_0.getLong(0);
/* 072 */           bnlj_isNull_9 = false;
/* 073 */           bnlj_value_9 = bnlj_value_8;
/* 074 */         }
/* 075 */         range_mutableStateArray_0[3].reset();
/* 076 */
/* 077 */         range_mutableStateArray_0[3].zeroOutNullBytes();
/* 078 */
/* 079 */         range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 080 */
/* 081 */         if (bnlj_isNull_9) {
/* 082 */           range_mutableStateArray_0[3].setNullAt(1);
/* 083 */         } else {
/* 084 */           range_mutableStateArray_0[3].write(1, bnlj_value_9);
/* 085 */         }
/* 086 */         append((range_mutableStateArray_0[3].getRow()).copy());
/* 087 */
/* 088 */       }
/* 089 */     }
/* 090 */
/* 091 */   }
/* 092 */
/* 093 */   private void initRange(int idx) {
/* 094 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 095 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(16L);
/* 096 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 097 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 098 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 099 */     long partitionEnd;
/* 100 */
/* 101 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 102 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 103 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 104 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 105 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 106 */     } else {
/* 107 */       range_nextIndex_0 = st.longValue();
/* 108 */     }
/* 109 */     range_batchEnd_0 = range_nextIndex_0;
/* 110 */
/* 111 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 112 */     .multiply(step).add(start);
/* 113 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 114 */       partitionEnd = Long.MAX_VALUE;
/* 115 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 116 */       partitionEnd = Long.MIN_VALUE;
/* 117 */     } else {
/* 118 */       partitionEnd = end.longValue();
/* 119 */     }
/* 120 */
/* 121 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 122 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 123 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 124 */     if (range_numElementsTodo_0 < 0) {
/* 125 */       range_numElementsTodo_0 = 0;
/* 126 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 127 */       range_numElementsTodo_0++;
/* 128 */     }
/* 129 */   }
/* 130 */
/* 131 */   protected void processNext() throws java.io.IOException {
/* 132 */     // initialize Range
/* 133 */     if (!range_initRange_0) {
/* 134 */       range_initRange_0 = true;
/* 135 */       initRange(partitionIndex);
/* 136 */     }
/* 137 */
/* 138 */     while (true) {
/* 139 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 140 */         long range_nextBatchTodo_0;
/* 141 */         if (range_numElementsTodo_0 > 1000L) {
/* 142 */           range_nextBatchTodo_0 = 1000L;
/* 143 */           range_numElementsTodo_0 -= 1000L;
/* 144 */         } else {
/* 145 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 146 */           range_numElementsTodo_0 = 0;
/* 147 */           if (range_nextBatchTodo_0 == 0) break;
/* 148 */         }
/* 149 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 150 */       }
/* 151 */
/* 152 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 153 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 154 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 155 */
/* 156 */         // common sub-expressions
/* 157 */
/* 158 */         bnlj_doConsume_0(range_value_0);
/* 159 */
/* 160 */         if (shouldStop()) {
/* 161 */           range_nextIndex_0 = range_value_0 + 1L;
/* 162 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 163 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 164 */           return;
/* 165 */         }
/* 166 */
/* 167 */       }
/* 168 */       range_nextIndex_0 = range_batchEnd_0;
/* 169 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 170 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 171 */       range_taskContext_0.killTaskIfInterrupted();
/* 172 */     }
/* 173 */   }
/* 174 */
/* 175 */ }
```

Closes #31931 from linzebing/code-left-right-outer.

Authored-by: linzebing <linzebing1995@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:11:57 +00:00
hezuojiao 39542bb81f [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled
### What changes were proposed in this pull request?

This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption.

### Why are the changes needed?
Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message:
```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message=
org.apache.spark.shuffle.FetchFailedException: Stream is corrupted
	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845)
	at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
	at java.io.DataInputStream.readInt(DataInputStream.java:387)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:494)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Stream is corrupted
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200)
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226)
	at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841)
	... 25 more

)
```

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

No

### How was this patch tested?

New tests.

Closes #31898 from hezuojiao/fetch_shuffle_in_batch.

Authored-by: hezuojiao <hezuojiao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 13:06:12 -07:00
tanel.kiis@gmail.com 51cf0cadea [SPARK-34812][SQL] RowNumberLike and RankLike should not be nullable
### What changes were proposed in this pull request?

Marked `RowNumberLike` and `RankLike` as not-nullable.

### Why are the changes needed?

`RowNumberLike` and `RankLike` SQL expressions never return null value. Marking them as non-nullable can have some performance benefits, because some optimizer rules apply only to non-nullable expressions

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

No

### How was this patch tested?

Did not find any existing tests on the nullability of aggregate functions.
Plan stability suite partially covers this.

Closes #31924 from tanelk/SPARK-34812_nullability.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 14:55:43 +00:00
woyumen4597 f44608a8c0 [SPARK-34800][SQL] Use fine-grained lock in SessionCatalog.tableExists
### What changes were proposed in this pull request?
Use fine-grained lock in SessionCatalog.tableExists, in order to lock currentDB variable rather than lock `tableExists` method which will block inner external catalog's behaviour.

### Why are the changes needed?
We have modified the underlying hive meta store which a different hive  database is placed in its own shard for performance. However, we found that the synchronized lock  limits the concurrency.

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

Closes #31891 from woyumen4597/SPARK-34800.

Authored-by: woyumen4597 <woyumen4597@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 09:03:46 +00:00
Terry Kim 7953fcdb56 [SPARK-34700][SQL] SessionCatalog's temporary view related APIs should take/return more concrete types
### What changes were proposed in this pull request?

Now that all the temporary views are wrapped with `TemporaryViewRelation`(#31273, #31652, and #31825), this PR proposes to update `SessionCatalog`'s APIs for temporary views to take or return more concrete types.

APIs that will take `TemporaryViewRelation` instead of `LogicalPlan`:
```
createTempView, createGlobalTempView, alterTempViewDefinition
```

APIs that will return `TemporaryViewRelation` instead of `LogicalPlan`:
```
getRawTempView, getRawGlobalTempView
```

APIs that will return `View` instead of `LogicalPlan`:
```
getTempView, getGlobalTempView, lookupTempView
```

### Why are the changes needed?

Internal refactoring to work with more concrete types.

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

No, this is internal refactoring.

### How was this patch tested?

Updated existing tests affected by the refactoring.

Closes #31906 from imback82/use_temporary_view_relation.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:17:54 +00:00
yi.wu e4bb97526c [SPARK-34089][CORE] HybridRowQueue should respect the configured memory mode
### What changes were proposed in this pull request?

This PR fixes the `HybridRowQueue ` to respect the configured memory mode.

Besides, this PR also refactored the constructor of `MemoryConsumer` to accept the memory mode explicitly.

### Why are the changes needed?

`HybridRowQueue` supports both onHeap and offHeap manipulation. But it inherited the wrong `MemoryConsumer` constructor, which hard-coded the memory mode to `onHeap`.

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

No. (Maybe yes in some cases where users can't complete the job before could complete successfully after the fix because of `HybridRowQueue` is able to spill under offHeap mode now. )

### How was this patch tested?

Updated the existing test to make it test both offHeap and onHeap modes.

Closes #31152 from Ngone51/fix-MemoryConsumer-memorymode.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:12:08 +00:00
HyukjinKwon ec70467d4d [SPARK-34815][SQL] Update CSVBenchmark
### What changes were proposed in this pull request?

This PR updates CSVBenchmark especially we have a fix like https://github.com/apache/spark/pull/31858 that could potentially improve the performance.

### Why are the changes needed?

To have the updated benchmark results.

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

No.

### How was this patch tested?

Manually ran the benchmark

Closes #31917 from HyukjinKwon/SPARK-34815.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 10:49:53 +03:00
Jungtaek Lim (HeartSaVioR) 121883b1a5 [SPARK-34383][SS] Optimize WAL commit phase via reducing cost of filesystem operations
### What changes were proposed in this pull request?

This PR proposes to optimize WAL commit phase via following changes:

* cache offset log to avoid FS get operation per batch
* just directly delete instead of employing FS list operation on purge

### Why are the changes needed?

There're inefficiency on WAL commit phase which can be easily optimized via using a small driver memory.

1. To provide the offset metadata to source side (via `source.commit()`), we read offset metadata for previous batch from file system, which is probably written by this driver in previous batches. Caching it into driver memory would reduce the get operation.
2. Spark calls purge against offset log & commit log per batch, which calls list operation. If the previous batch succeeded to purge, the current batch just needs to check one batch which can be simply done via direct delete operation, instead of calling list operation.

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

No.

### How was this patch tested?

Manually tested with additional debug log. (Verified that cache is used, cache keeps the size as 2, only one delete call is used instead of list call)

Did some experiment with simple rate to console query. (NOTE: wasn't done with master branch - tested against Spark 2.4.x, but WAL commit phase hasn't been changed AFAIK during these versions)

AWS S3 + S3 guard:

> before the patch

<img width="1075" alt="aws-before" src="https://user-images.githubusercontent.com/1317309/107108721-6cc54380-687d-11eb-8f10-b906b9d58397.png">

> after the patch

<img width="1071" alt="aws-after" src="https://user-images.githubusercontent.com/1317309/107108724-7189f780-687d-11eb-88da-26912ac15c85.png">

Azure:

> before the patch

<img width="1074" alt="azure-before" src="https://user-images.githubusercontent.com/1317309/107108726-75b61500-687d-11eb-8c06-9048fa10ff9a.png">

> after the patch

<img width="1069" alt="azure-after" src="https://user-images.githubusercontent.com/1317309/107108729-79e23280-687d-11eb-8d97-e7f3aeec51be.png">

Closes #31495 from HeartSaVioR/SPARK-34383.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
2021-03-22 08:47:07 +01:00
Cheng Su f8838fe82b [SPARK-34708][SQL] Code-gen for left semi/anti broadcast nested loop join (build right side)
### What changes were proposed in this pull request?

This PR is to add code-gen support for left semi / left anti BroadcastNestedLoopJoin (build side is right side). The execution code path for build left side cannot fit into whole stage code-gen framework, so only add the code-gen for build right side here.

Reference: the iterator (non-code-gen) code path is `BroadcastNestedLoopJoinExec.leftExistenceJoin()` with `BuildRight`.

### Why are the changes needed?

Improve query CPU performance.
Tested with a simple query:

```
val N = 20 << 20
val M = 1 << 4

val dim = broadcast(spark.range(M).selectExpr("id as k2"))
codegenBenchmark("left semi broadcast nested loop join", N) {
  park.range(N).selectExpr(s"id as k1").join(
    dim, col("k1") + 1 <= col("k2"), "left_semi")
}
```

Seeing 5x run time improvement:

```
Running benchmark: left semi broadcast nested loop join
  Running case: left semi broadcast nested loop join codegen off
  Stopped after 2 iterations, 6958 ms
  Running case: left semi broadcast nested loop join codegen on
  Stopped after 5 iterations, 3383 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.7
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
left semi broadcast nested loop join:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
--------------------------------------------------------------------------------------------------------------------------------
left semi broadcast nested loop join codegen off           3434           3479          65          6.1         163.7       1.0X
left semi broadcast nested loop join codegen on             672            677           5         31.2          32.1       5.1X
```

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

No.

### How was this patch tested?

Changed existing unit test in `ExistenceJoinSuite.scala` to cover all code paths:
* left semi/anti + empty right side + empty condition
* left semi/anti + non-empty right side + empty condition
* left semi/anti + right side + non-empty condition

Added unit test in `WholeStageCodegenSuite.scala` to make sure code-gen for broadcast nested loop join is taking effect, and test for multiple join case as well.

Example query:

```
val df1 = spark.range(4).select($"id".as("k1"))
val df2 = spark.range(3).select($"id".as("k2"))
df1.join(df2, $"k1" + 1 <= $"k2", "left_semi").explain("codegen")
```

Example generated code (`bnlj_doConsume_0` method):
This is for left semi join. The generated code for left anti join is mostly to be same as here, except L55 to be `if (bnlj_findMatchedRow_0 == false) {`.
```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) Project [id#0L AS k1#2L]
+- *(2) BroadcastNestedLoopJoin BuildRight, LeftSemi, ((id#0L + 1) <= k2#6L)
   :- *(2) Range (0, 4, step=1, splits=2)
   +- BroadcastExchange IdentityBroadcastMode, [id=#23]
      +- *(1) Project [id#4L AS k2#6L]
         +- *(1) Range (0, 3, step=1, splits=2)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage2(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=2
/* 006 */ final class GeneratedIteratorForCodegenStage2 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean range_initRange_0;
/* 010 */   private long range_nextIndex_0;
/* 011 */   private TaskContext range_taskContext_0;
/* 012 */   private InputMetrics range_inputMetrics_0;
/* 013 */   private long range_batchEnd_0;
/* 014 */   private long range_numElementsTodo_0;
/* 015 */   private InternalRow[] bnlj_buildRowArray_0;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[4];
/* 017 */
/* 018 */   public GeneratedIteratorForCodegenStage2(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */
/* 026 */     range_taskContext_0 = TaskContext.get();
/* 027 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
/* 028 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 029 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 030 */     bnlj_buildRowArray_0 = (InternalRow[]) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcastTerm */).value();
/* 031 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 032 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
/* 033 */
/* 034 */   }
/* 035 */
/* 036 */   private void bnlj_doConsume_0(long bnlj_expr_0_0) throws java.io.IOException {
/* 037 */     boolean bnlj_findMatchedRow_0 = false;
/* 038 */     for (int bnlj_arrayIndex_0 = 0; bnlj_arrayIndex_0 < bnlj_buildRowArray_0.length; bnlj_arrayIndex_0++) {
/* 039 */       UnsafeRow bnlj_buildRow_0 = (UnsafeRow) bnlj_buildRowArray_0[bnlj_arrayIndex_0];
/* 040 */
/* 041 */       long bnlj_value_1 = bnlj_buildRow_0.getLong(0);
/* 042 */
/* 043 */       long bnlj_value_3 = -1L;
/* 044 */
/* 045 */       bnlj_value_3 = bnlj_expr_0_0 + 1L;
/* 046 */
/* 047 */       boolean bnlj_value_2 = false;
/* 048 */       bnlj_value_2 = bnlj_value_3 <= bnlj_value_1;
/* 049 */       if (!(false || !bnlj_value_2))
/* 050 */       {
/* 051 */         bnlj_findMatchedRow_0 = true;
/* 052 */         break;
/* 053 */       }
/* 054 */     }
/* 055 */     if (bnlj_findMatchedRow_0 == true) {
/* 056 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 057 */
/* 058 */       // common sub-expressions
/* 059 */
/* 060 */       range_mutableStateArray_0[3].reset();
/* 061 */
/* 062 */       range_mutableStateArray_0[3].write(0, bnlj_expr_0_0);
/* 063 */       append((range_mutableStateArray_0[3].getRow()).copy());
/* 064 */
/* 065 */     }
/* 066 */
/* 067 */   }
/* 068 */
/* 069 */   private void initRange(int idx) {
/* 070 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 071 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
/* 072 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4L);
/* 073 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 074 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 075 */     long partitionEnd;
/* 076 */
/* 077 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 078 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 079 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 080 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 081 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 082 */     } else {
/* 083 */       range_nextIndex_0 = st.longValue();
/* 084 */     }
/* 085 */     range_batchEnd_0 = range_nextIndex_0;
/* 086 */
/* 087 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 088 */     .multiply(step).add(start);
/* 089 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 090 */       partitionEnd = Long.MAX_VALUE;
/* 091 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 092 */       partitionEnd = Long.MIN_VALUE;
/* 093 */     } else {
/* 094 */       partitionEnd = end.longValue();
/* 095 */     }
/* 096 */
/* 097 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 098 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 099 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 100 */     if (range_numElementsTodo_0 < 0) {
/* 101 */       range_numElementsTodo_0 = 0;
/* 102 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 103 */       range_numElementsTodo_0++;
/* 104 */     }
/* 105 */   }
/* 106 */
/* 107 */   protected void processNext() throws java.io.IOException {
/* 108 */     // initialize Range
/* 109 */     if (!range_initRange_0) {
/* 110 */       range_initRange_0 = true;
/* 111 */       initRange(partitionIndex);
/* 112 */     }
/* 113 */
/* 114 */     while (true) {
/* 115 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 116 */         long range_nextBatchTodo_0;
/* 117 */         if (range_numElementsTodo_0 > 1000L) {
/* 118 */           range_nextBatchTodo_0 = 1000L;
/* 119 */           range_numElementsTodo_0 -= 1000L;
/* 120 */         } else {
/* 121 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 122 */           range_numElementsTodo_0 = 0;
/* 123 */           if (range_nextBatchTodo_0 == 0) break;
/* 124 */         }
/* 125 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 126 */       }
/* 127 */
/* 128 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 129 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 130 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 131 */
/* 132 */         bnlj_doConsume_0(range_value_0);
/* 133 */
/* 134 */         if (shouldStop()) {
/* 135 */           range_nextIndex_0 = range_value_0 + 1L;
/* 136 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 137 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 138 */           return;
/* 139 */         }
/* 140 */
/* 141 */       }
/* 142 */       range_nextIndex_0 = range_batchEnd_0;
/* 143 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 144 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 145 */       range_taskContext_0.killTaskIfInterrupted();
/* 146 */     }
/* 147 */   }
/* 148 */
/* 149 */ }
```

Closes #31874 from c21/code-semi-anti.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 07:31:16 +00:00
Yuanjian Li 45235ac4bc [SPARK-34748][SS] Create a rule of the analysis logic for streaming write
### What changes were proposed in this pull request?
- Create a new rule `ResolveStreamWrite` for all analysis logic for streaming write.
- Add corresponding logical plans `WriteToStreamStatement` and `WriteToStream`.

### Why are the changes needed?
Currently, the analysis logic for streaming write is mixed in StreamingQueryManager. If we create a specific analyzer rule and separated logical plans, it should be helpful for further extension.

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

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

Closes #31842 from xuanyuanking/SPARK-34748.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 06:39:39 +00:00
Josh Soref f4de93efb0 [MINOR][SQL] Spelling: filters - PushedFilers
### What changes were proposed in this pull request?
Consistently correct the spelling of `PushedFilters`

### Why are the changes needed?
bersprockets noted that it's wrong

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

Technically, I think it does. Practically, neither Google nor GitHub show anyone using `pushedFilers` outside of forks (or the discussion about fixing it started at https://github.com/apache/spark/pull/30323#issuecomment-725568719)

### How was this patch tested?
None beyond CI in the previous PR

Closes #30678 from jsoref/spelling-filters.

Authored-by: Josh Soref <jsoref@users.noreply.github.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-03-22 08:00:12 +03:00
Dongjoon Hyun c5fd94f119 [SPARK-34772][TESTS][FOLLOWUP] Disable a test case using Hive 1.2.1 in Java9+ environment
### What changes were proposed in this pull request?

This PR aims to disable a new test case using Hive 1.2.1 from Java9+ test environment.

### Why are the changes needed?

[HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113) upgraded Datanucleus to 4.x at Hive 2.0. Datanucleus 3.x doesn't support Java9+.

**Java 9+ Environment**
```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] *** 1 TEST FAILED ***
[error] Failed: Total 1, Failed 1, Errors 0, Passed 0
[error] Failed tests:
[error] 	org.apache.spark.sql.hive.HiveSparkSubmitSuite
[error] (hive / Test / testOnly) sbt.TestsFailedException: Tests unsuccessful
[error] Total time: 328 s (05:28), completed Mar 21, 2021, 5:32:39 PM
```

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

Fix the UT in Java9+ environment.

### How was this patch tested?

Manually.

```
$ build/sbt "hive/testOnly *.HiveSparkSubmitSuite -- -z SPARK-34772" -Phive
...
[info] HiveSparkSubmitSuite:
[info] - SPARK-34772: RebaseDateTime loadRebaseRecords should use Spark classloader instead of context !!! CANCELED !!! (26 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:344)
```

Closes #31916 from dongjoon-hyun/SPARK-HiveSparkSubmitSuite.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 17:59:55 -07:00
Kousuke Saruta 94fd6cb0ce [SPARK-34636][FOLLOWUP][SQL] Fix an incompatible behavior of UnresolvedAttribute.sql
### What changes were proposed in this pull request?

This PR fixes an incompatible behavior introduced by #31754.
The problem is that quoted name parts represented as a string are given to the constructor of `UnresolvedAttribute` which takes single string parameter, `sql` method invocation against the `UnresolvedAttrribute` returns different result than before.

One example is ``` UnresolvedAttribute("`a.b`").sql ```. This  returned `a.b` before but it doesn't now.

See [this duscussion](https://github.com/apache/spark/pull/31754/files#r597181927) for more details.

### Why are the changes needed?

For compatibility.

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

No.

### How was this patch tested?

New assertion.

Closes #31885 from sarutak/followup-SPARK-34636.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-20 14:44:36 -07:00
Yuming Wang 908318f30d [SPARK-28220][SQL] Improve PropagateEmptyRelation to support join with false condition
### What changes were proposed in this pull request?

Improve `PropagateEmptyRelation` to support join with false condition. For example:
```sql
SELECT * FROM t1 LEFT JOIN t2 ON false
```

Before this pr:
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- BroadcastNestedLoopJoin BuildRight, LeftOuter, false
   :- FileScan parquet default.t1[a#4L]
   +- BroadcastExchange IdentityBroadcastMode, [id=#40]
      +- FileScan parquet default.t2[b#5L]
```

After this pr:
```
== Physical Plan ==
*(1) Project [a#4L, null AS b#5L]
+- *(1) ColumnarToRow
   +- FileScan parquet default.t1[a#4L]
```

### Why are the changes needed?

Avoid `BroadcastNestedLoopJoin` to improve query performance.

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

No.

### How was this patch tested?

Unit test.

Closes #31857 from wangyum/SPARK-28220.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-03-20 22:57:02 +08:00
Kent Yao 2cdedef2a0 [SPARK-34128][SQL] Suppress undesirable TTransportException warnings involved in THRIFT-4805
### What changes were proposed in this pull request?

Since Spark 3.0, the `libthrift` has been bumped up from 0.9.3 to 0.12.0.

Due to THRIFT-4805, The SparkThrift Server will print annoying TExceptions. For example, the current thrift server module test in Github action workflow outputs more than 200MB of data for this error only
```java
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

I checked the latest `hive-service-rpc` module in the maven center,  https://mvnrepository.com/artifact/org.apache.hive/hive-service-rpc/3.1.2.  It still uses the 0.9.3 version.

Unfortunately, I tried the newly released `libthrift 0.14.1`(w/o shading it), it breaks the metastore client side.

```scala
java.lang.NoSuchMethodError: org.apache.thrift.transport.TSocket.<init>(Ljava/lang/String;II)V
```
On the Thrift side, they just muted it see https://issues.apache.org/jira/browse/THRIFT-4805

So in this PR, I add a filter to suppress the warning

### Why are the changes needed?

if the log is too large, the Github action might truncate it. We need to reduce useless output.

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

no

### How was this patch tested?

```build/sbt "hive-thriftserver/testOnly *ThriftServerQueryTestSuite" -Phive-thriftserver``` locally

#### before

```java
[info] - count.sql (1 second, 537 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
14:09:53.233 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - group-analytics.sql (4 seconds, 282 milliseconds)

[info] - csv-functions.sql (400 milliseconds)
14:09:24.234 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - datetime-formatting-invalid.sql (349 milliseconds)
14:09:26.544 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - except.sql (2 seconds, 309 milliseconds)
14:09:27.782 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
[info] - string-functions.sql (1 second, 237 milliseconds)
14:09:27.835 WARN org.apache.spark.sql.execution.datasources.DataSource: All paths were ignored:

14:09:29.266 ERROR org.apache.thrift.server.TThreadPoolServer: Thrift error occurred during processing of message.
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:374)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:451)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:433)
	at org.apache.thrift.transport.TSaslServerTransport.read(TSaslServerTransport.java:43)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
	at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:425)
	at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:321)
	at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:225)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:27)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

```

#### after

```java

[info] - null-propagation.sql (181 milliseconds)
[info] - operators.sql (1 second, 772 milliseconds)
[info] - change-column.sql (241 milliseconds)
[info] - count.sql (1 second, 665 milliseconds)
[info] - decimalArithmeticOperations.sql !!! IGNORED !!!
[info] - group-analytics.sql (3 seconds, 926 milliseconds)
[info] - inline-table.sql (247 milliseconds)
[info] - comparator.sql (223 milliseconds)
[info] - show-tblproperties.sql (148 milliseconds)
[info] - timezone.sql (105 milliseconds)
[info] - parse-schema-string.sql (193 milliseconds)
```

Closes #31895 from yaooqinn/SPARK-34128-2.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-19 21:15:28 -07:00
Cheng Su 2ff0032e01 [SPARK-34796][SQL] Initialize counter variable for LIMIT code-gen in doProduce()
### What changes were proposed in this pull request?

This PR is to fix the LIMIT code-gen bug in https://issues.apache.org/jira/browse/SPARK-34796, where the counter variable from `BaseLimitExec` is not initialized but used in code-gen. This is because the limit counter variable will be used in upstream operators (LIMIT's child plan, e.g. `ColumnarToRowExec` operator for early termination), but in the same stage, there can be some operators doing the shortcut and not calling `BaseLimitExec`'s `doConsume()`, e.g. [HashJoin.codegenInner](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L402). So if we have query that `LocalLimit - BroadcastHashJoin - FileScan` in the same stage, the whole stage code-gen compilation will be failed.

Here is an example:

```
  test("failed limit query") {
    withTable("left_table", "empty_right_table", "output_table") {
      spark.range(5).toDF("k").write.saveAsTable("left_table")
      spark.range(0).toDF("k").write.saveAsTable("empty_right_table")

      withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
        spark.sql("CREATE TABLE output_table (k INT) USING parquet")
        spark.sql(
          s"""
             |INSERT INTO TABLE output_table
             |SELECT t1.k FROM left_table t1
             |JOIN empty_right_table t2
             |ON t1.k = t2.k
             |LIMIT 3
             |""".stripMargin)
      }
    }
  }
```

Query plan:

```
Execute InsertIntoHadoopFsRelationCommand file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table, false, Parquet, Map(path -> file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table), Append, CatalogTable(
Database: default
Table: output_table
Created Time: Thu Mar 18 21:46:26 PDT 2021
Last Access: UNKNOWN
Created By: Spark 3.2.0-SNAPSHOT
Type: MANAGED
Provider: parquet
Location: file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQuerySuite/output_table
Schema: root
 |-- k: integer (nullable = true)
), org.apache.spark.sql.execution.datasources.InMemoryFileIndexb25d08b, [k]
+- *(3) Project [ansi_cast(k#228L as int) AS k#231]
   +- *(3) GlobalLimit 3
      +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [id=#179]
         +- *(2) LocalLimit 3
            +- *(2) Project [k#228L]
               +- *(2) BroadcastHashJoin [k#228L], [k#229L], Inner, BuildRight, false
                  :- *(2) Filter isnotnull(k#228L)
                  :  +- *(2) ColumnarToRow
                  :     +- FileScan parquet default.left_table[k#228L] Batched: true, DataFilters: [isnotnull(k#228L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
                  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#173]
                     +- *(1) Filter isnotnull(k#229L)
                        +- *(1) ColumnarToRow
                           +- FileScan parquet default.empty_right_table[k#229L] Batched: true, DataFilters: [isnotnull(k#229L)], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/Users/chengsu/spark/sql/core/spark-warehouse/org.apache.spark.sq..., PartitionFilters: [], PushedFilters: [IsNotNull(k)], ReadSchema: struct<k:bigint>
```

Codegen failure - https://gist.github.com/c21/ea760c75b546d903247582be656d9d66 .

The uninitialized variable `_limit_counter_1` from `LocalLimitExec` is referenced in `ColumnarToRowExec`, but `BroadcastHashJoinExec` does not call `LocalLimitExec.doConsume()` to initialize the counter variable.

The fix is to move the counter variable initialization to `doProduce()`, as in whole stage code-gen framework, `doProduce()` will definitely be called if upstream operators `doProduce()`/`doConsume()` is called.

Note: this only happens in AQE disabled case, because we have an AQE optimization rule [EliminateUnnecessaryJoin](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/EliminateUnnecessaryJoin.scala#L69) to change the whole query to an empty `LocalRelation` if inner join broadcast side is empty with AQE enabled.

### Why are the changes needed?

Fix query failure.

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

No.

### How was this patch tested?

Added unit test in `SQLQuerySuite.scala`.

Closes #31892 from c21/limit-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:20:52 +09:00
tanel.kiis@gmail.com 620cae098c [SPARK-33122][SQL] Remove redundant aggregates in the Optimzier
### What changes were proposed in this pull request?

Added optimizer rule `RemoveRedundantAggregates`. It removes redundant aggregates from a query plan. A redundant aggregate is an aggregate whose only goal is to keep distinct values, while its parent aggregate would ignore duplicate values.

The affected part of the query plan for TPCDS q87:

Before:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#785]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
            +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                  +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                     +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#724]
                        +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
                           +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                              :- ...
```

After:
```
== Physical Plan ==
*(26) HashAggregate(keys=[], functions=[count(1)])
+- Exchange SinglePartition, true, [id=#751]
   +- *(25) HashAggregate(keys=[], functions=[partial_count(1)])
      +- *(25) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
         +- Exchange hashpartitioning(c_last_name#61, c_first_name#60, d_date#26, 5), true, [id=#694]
            +- *(24) HashAggregate(keys=[c_last_name#61, c_first_name#60, d_date#26], functions=[])
               +- SortMergeJoin [coalesce(c_last_name#61, ), isnull(c_last_name#61), coalesce(c_first_name#60, ), isnull(c_first_name#60), coalesce(d_date#26, 0), isnull(d_date#26)], [coalesce(c_last_name#221, ), isnull(c_last_name#221), coalesce(c_first_name#220, ), isnull(c_first_name#220), coalesce(d_date#186, 0), isnull(d_date#186)], LeftAnti
                  :- ...
```

### Why are the changes needed?

Performance improvements - few TPCDS queries have these kinds of duplicate aggregates.

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

No

### How was this patch tested?

UT

Benchmarks (sf=5):

OpenJDK 64-Bit Server VM 1.8.0_265-b01 on Linux 5.8.13-arch1-1
Intel(R) Core(TM) i5-6500 CPU  3.20GHz

| Query | Before  | After | Speedup |
| ------| ------- | ------| ------- |
| q14a | 44s | 44s | 1x |
| q14b | 41s | 41s | 1x |
| q38  | 6.5s | 5.9s | 1.1x |
| q87  | 7.2s | 6.8s | 1.1x |
| q14a-v2.7 | 55s | 53s | 1x |

Closes #30018 from tanelk/SPARK-33122.

Lead-authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Co-authored-by: Tanel Kiis <tanel.kiis@reach-u.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-20 11:16:39 +09:00