Commit graph

10148 commits

Author SHA1 Message Date
allisonwang-db 2e3aa2f023 [SPARK-32861][SQL] GenerateExec should require column ordering
### What changes were proposed in this pull request?
This PR updates the `RemoveRedundantProjects` rule to make `GenerateExec` require column ordering.

### Why are the changes needed?
`GenerateExec` was originally considered as a node that does not require column ordering. However, `GenerateExec` binds its input rows directly with its `requiredChildOutput` without using the child's output schema.
In `doExecute()`:
```scala
val proj = UnsafeProjection.create(output, output)
```
In `doConsume()`:
```scala
val values = if (requiredChildOutput.nonEmpty) {
  input
} else {
  Seq.empty
}
```
In this case, changing input column ordering will result in `GenerateExec` binding the wrong schema to the input columns. For example, if we do not require child columns to be ordered, the `requiredChildOutput` [a, b, c] will directly bind to the schema of the input columns [c, b, a], which is incorrect:
```
GenerateExec explode(array(a, b, c)), [a, b, c], false, [d]
  HashAggregate(keys=[a, b, c], functions=[], output=[c, b, a])
    ...
```

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

### How was this patch tested?
Unit test

Closes #29734 from allisonwang-db/generator.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-16 06:05:35 +00:00
Tanel Kiis 6051755bfe [SPARK-32688][SQL][TEST] Add special values to LiteralGenerator for float and double
### What changes were proposed in this pull request?

The `LiteralGenerator` for float and double datatypes was supposed to yield special values (NaN, +-inf) among others, but the `Gen.chooseNum` method does not yield values that are outside the defined range. The `Gen.chooseNum` for a wide range of floats and doubles does not yield values in the "everyday" range as stated in https://github.com/typelevel/scalacheck/issues/113 .

There is an similar class `RandomDataGenerator` that is used in some other tests. Added `-0.0` and `-0.0f` as special values to there too.

These changes revealed an inconsistency with the equality check between `-0.0` and `0.0`.

### Why are the changes needed?

The `LiteralGenerator` is mostly used in the `checkConsistencyBetweenInterpretedAndCodegen` method in `MathExpressionsSuite`. This change would have caught the bug fixed in #29495 .

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

No

### How was this patch tested?

Locally reverted #29495 and verified that the existing test cases caught the bug.

Closes #29515 from tanelk/SPARK-32688.

Authored-by: Tanel Kiis <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-16 12:13:15 +09:00
HyukjinKwon b46c7302db [SPARK-32704][SQL][TESTS][FOLLOW-UP] Check any physical rule instead of a specific rule in the test
### What changes were proposed in this pull request?

This PR only checks if there's any physical rule runs instead of a specific rule. This is rather just a trivial fix to make the tests more robust.

In fact, I faced a test failure from a in-house fork that applies a different physical rule that makes `CollapseCodegenStages` ineffective.

### Why are the changes needed?

To make the test more robust by unrelated changes.

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

No, test-only

### How was this patch tested?

Manually tested. Jenkins tests should pass.

Closes #29766 from HyukjinKwon/SPARK-32704.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-16 12:06:57 +09:00
HyukjinKwon 108c4c8fdc [SPARK-32481][SQL][TESTS][FOLLOW-UP] Skip the test if trash directory cannot be created
### What changes were proposed in this pull request?

This PR skips the test if trash directory cannot be created. It is possible that the trash directory cannot be created, for example, by permission. And the test fails below:

```
- SPARK-32481 Move data to trash on truncate table if enabled *** FAILED *** (154 milliseconds)
  fs.exists(trashPath) was false (DDLSuite.scala:3184)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
  at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
```

### Why are the changes needed?

To make the tests pass independently.

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

No, test-only.

### How was this patch tested?

Manually tested.

Closes #29759 from HyukjinKwon/SPARK-32481.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-16 08:11:43 +09:00
ulysses 888b343587 [SPARK-32827][SQL] Add spark.sql.maxMetadataStringLength config
### What changes were proposed in this pull request?

Add a new config `spark.sql.maxMetadataStringLength`. This config aims to limit metadata value length, e.g. file location.

### Why are the changes needed?

Some metadata have been abbreviated by `...` when I tried to add some test in `SQLQueryTestSuite`. We need to replace such value to `notIncludedMsg`. That caused we can't replace that like location value by `className` since the `className` has been abbreviated.

Here is a case:
```
CREATE table  explain_temp1 (key int, val int) USING PARQUET;

EXPLAIN EXTENDED SELECT sum(distinct val) FROM explain_temp1;

-- ignore parsed,analyzed,optimized
-- The output like
== Physical Plan ==
*HashAggregate(keys=[], functions=[sum(distinct cast(val#x as bigint)#xL)], output=[sum(DISTINCT val)#xL])
+- Exchange SinglePartition, true, [id=#x]
   +- *HashAggregate(keys=[], functions=[partial_sum(distinct cast(val#x as bigint)#xL)], output=[sum#xL])
      +- *HashAggregate(keys=[cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
         +- Exchange hashpartitioning(cast(val#x as bigint)#xL, 4), true, [id=#x]
            +- *HashAggregate(keys=[cast(val#x as bigint) AS cast(val#x as bigint)#xL], functions=[], output=[cast(val#x as bigint)#xL])
               +- *ColumnarToRow
                  +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/home/runner/work/spark/spark/sql/core/spark-warehouse/org.apache.spark.sq...], PartitionFilters: ...
```

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

No, a new config.

### How was this patch tested?

new test.

Closes #29688 from ulysses-you/SPARK-32827.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 14:11:30 +00:00
Kent Yao 316242b768 [SPARK-32874][SQL][TEST] Enhance result set meta data check for execute statement operation with thrift server
### What changes were proposed in this pull request?

This PR adds test cases for the result set metadata checking for Spark's `ExecuteStatementOperation` to make the JDBC API more future-proofing because any server-side change may affect the client compatibility.

### Why are the changes needed?

add test to prevent potential silent behavior change for JDBC users.

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

NO, test only
### How was this patch tested?

add new test

Closes #29746 from yaooqinn/SPARK-32874.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 13:07:03 +00:00
herman c8baab1a1f [SPARK-32879][SQL] Refactor SparkSession initial options
### What changes were proposed in this pull request?
This PR refactors the way we propagate the options from the `SparkSession.Builder` to the` SessionState`. This currently done via a mutable map inside the SparkSession. These setting settings are then applied **after** the Session. This is a bit confusing when you expect something to be set when constructing the `SessionState`. This PR passes the options as a constructor parameter to the `SessionStateBuilder` and this will set the options when the configuration is created.

### Why are the changes needed?
It makes it easier to reason about the configurations set in a SessionState than before. We recently had an incident where someone was using `SparkSessionExtensions` to create a planner rule that relied on a conf to be set. While this is in itself probably incorrect usage, it still illustrated this somewhat funky behavior.

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

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

Closes #29752 from hvanhovell/SPARK-32879.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-15 06:24:54 +00:00
Dongjoon Hyun d8a0d85692 [SPARK-32884][TESTS] Mark TPCDSQuery*Suite as ExtendedSQLTest
### What changes were proposed in this pull request?

This PR aims to mark the following suite as `ExtendedSQLTest` to reduce GitHub Action test time.
- TPCDSQuerySuite
- TPCDSQueryANSISuite
- TPCDSQueryWithStatsSuite

### Why are the changes needed?

Currently, the longest GitHub Action task is `Build and test / Build modules: sql - other tests` with `1h 57m 10s` while `Build and test / Build modules: sql - slow tests` takes `42m 20s`. With this PR, we can move the workload from `other tests` to `slow tests` task and reduce the total waiting time about 7 ~ 8 minutes.

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

No. This is a test-only change.

### How was this patch tested?

Pass the GitHub Action with the reduced running time.

Closes #29755 from dongjoon-hyun/SPARK-SLOWTEST.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-15 14:38:01 +09:00
Kousuke Saruta 4fac6d501a [SPARK-32871][BUILD] Append toMap to Map#filterKeys if the result of filter is concatenated with another Map for Scala 2.13
### What changes were proposed in this pull request?

This PR appends `toMap` to `Map` instances with `filterKeys` if such maps is to be concatenated with another maps.

### Why are the changes needed?

As of Scala 2.13, Map#filterKeys return a MapView, not the original Map type.
This can cause compile error.
```
/sql/DataFrameReader.scala:279: type mismatch;
[error]  found   : Iterable[(String, String)]
[error]  required: java.util.Map[String,String]
[error] Error occurred in an application involving default arguments.
[error]       val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava)
```

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

No.

### How was this patch tested?

Compile passed with the following command.
`build/mvn -Pscala-2.13 -Phive -Phive-thriftserver -Pyarn -Pkubernetes -DskipTests test-compile`

Closes #29742 from sarutak/fix-filterKeys-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-15 09:27:47 +09:00
HyukjinKwon 0696f04672 [SPARK-32876][SQL] Change default fallback versions to 3.0.1 and 2.4.7 in HiveExternalCatalogVersionsSuite
### What changes were proposed in this pull request?

The Jenkins job fails to get the versions. This was fixed by adding temporary fallbacks at https://github.com/apache/spark/pull/28536.
This still doesn't work without the temporary fallbacks. See https://github.com/apache/spark/pull/29694

This PR adds new fallbacks since 2.3 is EOL and Spark 3.0.1 and 2.4.7 are released.

### Why are the changes needed?

To test correctly in Jenkins.

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

No, dev-only

### How was this patch tested?

Jenkins and GitHub Actions builds should test.

Closes #29748 from HyukjinKwon/SPARK-32876.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-14 13:54:21 -07:00
tanel.kiis@gmail.com 7a17158a4d [SPARK-32868][SQL] Add more order irrelevant aggregates to EliminateSorts
### What changes were proposed in this pull request?

Mark `BitAggregate` as order irrelevant in `EliminateSorts`.

### Why are the changes needed?

Performance improvements in some queries

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

No

### How was this patch tested?

Generalized an existing UT

Closes #29740 from tanelk/SPARK-32868.

Authored-by: tanel.kiis@gmail.com <tanel.kiis@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-14 22:52:33 +09:00
Yuanjian Li 5e825482d7 [SPARK-32844][SQL] Make DataFrameReader.table take the specified options for datasource v1
### What changes were proposed in this pull request?
Make `DataFrameReader.table` take the specified options for datasource v1.

### Why are the changes needed?
Keep the same behavior of v1/v2 datasource, the v2 fix has been done in SPARK-32592.

### Does this PR introduce _any_ user-facing change?
Yes. The DataFrameReader.table will take the specified options. Also, if there are the same key and value exists in specified options and table properties, an exception will be thrown.

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

Closes #29712 from xuanyuanking/SPARK-32844.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-14 09:20:24 +00:00
Cheng Su 978f531010 [SPARK-32854][SS] Minor code and doc improvement for stream-stream join
### What changes were proposed in this pull request?

Several minor code and documentation improvement for stream-stream join. Specifically:

* Remove extending from `SparkPlan`, as extending from `BinaryExecNode` is enough.
* Return `left/right.outputPartitioning` for `Left/RightOuter` in `outputPartitioning`, as the `PartitioningCollection` wrapper is unnecessary (similar to batch joins `ShuffledHashJoinExec`, `SortMergeJoinExec`).
*  Avoid per-row check for join type (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala#L486-L492), by creating the method before the loop of reading rows (`generateFilteredJoinedRow` in `storeAndJoinWithOtherSide`). Similar optimization (i.e. create auxiliary method/variable per different join type before the iterator of input rows) has been done in batch join world (`SortMergeJoinExec`, `ShuffledHashJoinExec`).
* Minor fix for comment/indentation for better readability.

### Why are the changes needed?

Minor optimization to avoid per-row unnecessary work (this probably can be optimized away by compiler, but we can do a better join to avoid it at the first place). And other comment/indentation fix to have better code readability for future developers.

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

No.

### How was this patch tested?

Existing tests in `StreamingJoinSuite.scala` as no new logic is introduced.

Closes #29724 from c21/streaming.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-14 08:49:51 +00:00
Kousuke Saruta b121f0d459 [SPARK-32873][BUILD] Fix code which causes error when build with sbt and Scala 2.13
### What changes were proposed in this pull request?

This PR fix code which causes error when build with sbt and Scala 2.13 like as follows.
```
[error] [warn] /home/kou/work/oss/spark-scala-2.13/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala:251: method with a single empty parameter list overrides method without any parameter list
[error] [warn]   override def hasNext(): Boolean = requestOffset < part.untilOffset
[error] [warn]
[error] [warn] /home/kou/work/oss/spark-scala-2.13/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala:294: method with a single empty parameter list overrides method without any parameter list
[error] [warn]   override def hasNext(): Boolean = okNext
```

More specifically, what this PR fixes are

* Methods which has an empty parameter list and overrides an method which has no parameter list.
```
override def hasNext(): Boolean = okNext
```

* Methods which has no parameter list and overrides an method which has an empty parameter list.
```
      override def next: (Int, Double) = {
```

* Infix operator expression that the operator wraps.
```
    3L * math.min(k, numFeatures) * math.min(k, numFeatures)
    3L * math.min(k, numFeatures) * math.min(k, numFeatures) +
    + math.max(math.max(k, numFeatures), 4L * math.min(k, numFeatures)
      math.max(math.max(k, numFeatures), 4L * math.min(k, numFeatures) *
    * math.min(k, numFeatures) + 4L * math.min(k, numFeatures))
```

### Why are the changes needed?

For building Spark with sbt and Scala 2.13.

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

No.

### How was this patch tested?

After this change and #29742 applied, compile passed with the following command.
```
build/sbt -Pscala-2.13  -Phive -Phive-thriftserver -Pyarn -Pkubernetes compile test:compile
```

Closes #29745 from sarutak/fix-code-for-sbt-and-spark-2.13.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-14 15:34:58 +09:00
Chao Sun a6d6ea3efe [SPARK-32802][SQL] Avoid using SpecificInternalRow in RunLengthEncoding#Encoder
### What changes were proposed in this pull request?

Currently `RunLengthEncoding#Encoder` uses `SpecificInternalRow` as a holder for the current value when calculating compression stats and doing the actual compression. It calls `ColumnType.copyField` and `ColumnType.getField` on the internal row which incurs extra cost comparing to directly operating on the internal type. This proposes to replace the `SpecificInternalRow` with `T#InternalType` to avoid the extra cost.

### Why are the changes needed?

Operating on `SpecificInternalRow` carries certain cost and negatively impact performance when using `RunLengthEncoding` for compression.

With the change I see some improvements through `CompressionSchemeBenchmark`:

```diff
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 BOOLEAN Encode:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    1              1           0      51957.0           0.0       1.0X
-RunLengthEncoding(2.502)                            549            555           9        122.2           8.2       0.0X
-BooleanBitSet(0.125)                                296            301           3        226.6           4.4       0.0X
+PassThrough(1.000)                                    2              2           0      42985.4           0.0       1.0X
+RunLengthEncoding(2.517)                            487            500          10        137.7           7.3       0.0X
+BooleanBitSet(0.125)                                348            353           4        192.8           5.2       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 SHORT Encode (Lower Skew):                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    3              3           0      22779.9           0.0       1.0X
-RunLengthEncoding(1.520)                           1186           1192           9         56.6          17.7       0.0X
+PassThrough(1.000)                                    3              4           0      21216.6           0.0       1.0X
+RunLengthEncoding(1.493)                            882            931          50         76.1          13.1       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 SHORT Encode (Higher Skew):               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    3              4           0      21352.2           0.0       1.0X
-RunLengthEncoding(2.009)                           1173           1175           3         57.2          17.5       0.0X
+PassThrough(1.000)                                    3              3           0      22388.6           0.0       1.0X
+RunLengthEncoding(2.015)                            924            941          23         72.6          13.8       0.0X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 INT Encode (Lower Skew):                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    9             10           1       7410.1           0.1       1.0X
-RunLengthEncoding(1.000)                           1499           1502           4         44.8          22.3       0.0X
-DictionaryEncoding(0.500)                           621            630          11        108.0           9.3       0.0X
-IntDelta(0.250)                                     134            149          10        502.0           2.0       0.1X
+PassThrough(1.000)                                    9             10           1       7575.9           0.1       1.0X
+RunLengthEncoding(1.002)                            952            966          12         70.5          14.2       0.0X
+DictionaryEncoding(0.500)                           561            567           6        119.7           8.4       0.0X
+IntDelta(0.250)                                     129            134           3        521.9           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 INT Encode (Higher Skew):                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                    9             10           1       7668.3           0.1       1.0X
-RunLengthEncoding(1.332)                           1561           1685         175         43.0          23.3       0.0X
-DictionaryEncoding(0.501)                           616            642          21        108.9           9.2       0.0X
-IntDelta(0.250)                                     126            131           2        533.4           1.9       0.1X
+PassThrough(1.000)                                    9             10           1       7494.1           0.1       1.0X
+RunLengthEncoding(1.336)                            974            987          13         68.9          14.5       0.0X
+DictionaryEncoding(0.501)                           709            719          10         94.6          10.6       0.0X
+IntDelta(0.250)                                     127            132           4        528.4           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 LONG Encode (Lower Skew):                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   18             19           1       3803.0           0.3       1.0X
-RunLengthEncoding(0.754)                           1526           1540          20         44.0          22.7       0.0X
-DictionaryEncoding(0.250)                           735            759          33         91.3          11.0       0.0X
-LongDelta(0.125)                                    126            129           2        530.8           1.9       0.1X
+PassThrough(1.000)                                   19             21           1       3543.5           0.3       1.0X
+RunLengthEncoding(0.747)                           1049           1058          12         63.9          15.6       0.0X
+DictionaryEncoding(0.250)                           620            634          17        108.2           9.2       0.0X
+LongDelta(0.125)                                    129            132           2        520.1           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 LONG Encode (Higher Skew):                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   18             20           1       3705.4           0.3       1.0X
-RunLengthEncoding(1.002)                           1665           1669           6         40.3          24.8       0.0X
-DictionaryEncoding(0.251)                           890            901          11         75.4          13.3       0.0X
-LongDelta(0.125)                                    125            130           3        537.2           1.9       0.1X
+PassThrough(1.000)                                   18             20           2       3726.8           0.3       1.0X
+RunLengthEncoding(0.999)                           1076           1077           2         62.4          16.0       0.0X
+DictionaryEncoding(0.251)                           904            919          19         74.3          13.5       0.0X
+LongDelta(0.125)                                    125            131           4        536.5           1.9       0.1X

 OpenJDK 64-Bit Server VM 11.0.8+10-LTS on Mac OS X 10.15.5
 Intel(R) Core(TM) i9-9880H CPU  2.30GHz
 STRING Encode:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000)                                   27             30           2       2497.1           0.4       1.0X
-RunLengthEncoding(0.892)                           3443           3587         204         19.5          51.3       0.0X
-DictionaryEncoding(0.167)                          2286           2290           6         29.4          34.1       0.0X
+PassThrough(1.000)                                   28             31           2       2430.2           0.4       1.0X
+RunLengthEncoding(0.889)                           1798           1800           3         37.3          26.8       0.0X
+DictionaryEncoding(0.167)                          1956           1959           4         34.3          29.1       0.0X
```

In the above diff, new results are with changes in this PR. It can be seen that encoding performance has improved quite a lot especially for string type.

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

No.

### How was this patch tested?

Relies on existing unit tests.

Closes #29654 from sunchao/SPARK-32802.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 22:19:30 -07:00
Chao Sun 3d08084022 [SPARK-24994][SQL] Add UnwrapCastInBinaryComparison optimizer to simplify literal types
### What changes were proposed in this pull request?

Currently, in cases like the following:
```sql
SELECT * FROM t WHERE age < 40
```
where `age` is of short type, Spark won't be able to simplify this and can only generate filter `cast(age, int) < 40`. This won't get pushed down to datasources and therefore is not optimized.

This PR proposes a optimizer rule to improve this when the following constraints are satisfied:
 - input expression is binary comparisons when one side is a cast operation and another is a literal.
 - both the cast child expression and literal are of integral type (i.e., byte, short, int or long)

When this is true, it tries to do several optimizations to either simplify the expression or move the cast to the literal side, so
result filter for the above case becomes `age < cast(40 as smallint)`. This is better since the cast can be optimized away later and the filter can be pushed down to data sources.

This PR follows a similar effort in Presto (https://prestosql.io/blog/2019/05/21/optimizing-the-casts-away.html). Here we only handles integral types but plan to extend to other types as follow-ups.

### Why are the changes needed?

As mentioned in the previous section, when cast is not optimized, it cannot be pushed down to data sources which can lead
to unnecessary IO and therefore longer job time and waste of resources. This helps to improve that.

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

No.

### How was this patch tested?

Added unit tests for both the optimizer rule and filter pushdown on datasource level for both Orc and Parquet.

Closes #29565 from sunchao/SPARK-24994.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 21:34:35 -07:00
Karol Chmist 3be552ccc8 [SPARK-30090][SHELL] Adapt Spark REPL to Scala 2.13
### What changes were proposed in this pull request?

This is an attempt to adapt Spark REPL to Scala 2.13.

It is based on a [scala-2.13 branch](https://github.com/smarter/spark/tree/scala-2.13) made by smarter.

I had to set Scala version to 2.13 in some places, and to adapt some other modules, before I could start working on the REPL itself. These are separate commits on the branch that probably would be fixed beforehand, and thus dropped before the merge of this PR.

I couldn't find a way to run the initialization code with existing REPL classes in Scala 2.13.2, so I [modified REPL in Scala](e9cc0dd547) to make it work. With this modification I managed to run Spark Shell, along with the units tests passing, which is good news.

The bad news is that it requires an upstream change in Scala, which must be accepted first. I'd be happy to change it if someone points a way to do it differently. If not, I'd propose a PR in Scala to introduce `ILoop.internalReplAutorunCode`.

### Why are the changes needed?

REPL in Scala changed quite a lot, so current version of Spark REPL needed to be adapted.

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

In the previous version of `SparkILoop`, a lot of Scala's `ILoop` code was [overridden and duplicated](2bc7b75537) to make the welcome message a bit more pleasant. In this PR, the message is in a bit different order, but it's still acceptable IMHO.

Before this PR:
```
20/05/15 15:32:39 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:39 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:39 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/05/15 15:32:45 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
Spark context Web UI available at http://192.168.1.28:4041
Spark context available as 'sc' (master = local[*], app id = local-1589549565502).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.1-SNAPSHOT
      /_/

Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.

scala>
```

With this PR:
```
20/05/15 15:32:15 WARN Utils: Your hostname, hermes resolves to a loopback address: 127.0.1.1; using 192.168.1.28 instead (on interface enp0s31f6)
20/05/15 15:32:15 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
20/05/15 15:32:15 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.13.2-20200422-211118-706ef1b (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
Type in expressions to have them evaluated.
Type :help for more information.
Spark context Web UI available at http://192.168.1.28:4040
Spark context available as 'sc' (master = local[*], app id = local-1589549541259).
Spark session available as 'spark'.

scala>
```

It seems that currently the welcoming message is still an improvement from [the original ticket](https://issues.apache.org/jira/browse/SPARK-24785), albeit in a different order. As a bonus, some fragile code duplication was removed.

### How was this patch tested?

Existing tests pass in `repl`module. The REPL runs in a terminal and the following code executed correctly:

```
scala> spark.range(1000 * 1000 * 1000).count()
val res0: Long = 1000000000
```

Closes #28545 from karolchmist/scala-2.13-repl.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-12 18:15:15 -05:00
sandeep.katta 2009f95340 [SPARK-32779][SQL][FOLLOW-UP] Delete Unused code
### What changes were proposed in this pull request?
Follow-up PR as per the review comments in [29649](8d45542e91 (r487140171))

### Why are the changes needed?
Delete the un used code

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

### How was this patch tested?
Existing UT

Closes #29736 from sandeep-katta/deadlockfollowup.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-12 13:22:54 -07:00
Takeshi Yamamuro 4269c2c252 [SPARK-32851][SQL][TEST] Tests should fail if errors happen when generating projection code
### What changes were proposed in this pull request?

This PR intends to set `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE` in test spark context so that tests can fail if errors happen when generating expr code.

### Why are the changes needed?

I noticed that the code generation of `SafeProjection` failed in the existing test (https://issues.apache.org/jira/browse/SPARK-32828) but it passed because `FALLBACK` was set at `CODEGEN_FACTORY_MODE` (by default) in `SharedSparkSession`. To get aware of these failures quickly, I think its worth setting `CODEGEN_ONLY` at `CODEGEN_FACTORY_MODE`.

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

No.

### How was this patch tested?

Existing tests.

Closes #29721 from maropu/ExprCodegenTest.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-12 08:42:07 +09:00
Dongjoon Hyun b4be6a6d12 [SPARK-32845][SS][TESTS] Add sinkParameter to check sink options robustly in DataStreamReaderWriterSuite
### What changes were proposed in this pull request?

This PR aims to add `sinkParameter`  to check sink options robustly and independently in DataStreamReaderWriterSuite

### Why are the changes needed?

`LastOptions.parameters` is designed to catch three cases: `sourceSchema`, `createSource`, `createSink`. However, `StreamQuery.stop` invokes `queryExecutionThread.join`, `runStream`, `createSource` immediately and reset the stored options by `createSink`.

To catch `createSink` options, currently, the test suite is trying a workaround pattern. However, we observed a flakiness in this pattern sometimes. If we split `createSink` option separately, we don't need this workaround and can eliminate this flakiness.

```scala
val query = df.writeStream.
   ...
   .start()
assert(LastOptions.paramters(..))
query.stop()
```

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

No. This is a test-only change.

### How was this patch tested?

Pass the newly updated test case.

Closes #29730 from dongjoon-hyun/SPARK-32845.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-11 11:48:34 -07:00
Peter Toth 94cac5978c [SPARK-32730][SQL][FOLLOW-UP] Improve LeftAnti SortMergeJoin right side buffering
### What changes were proposed in this pull request?

This is a follow-up to https://github.com/apache/spark/pull/29572.

LeftAnti SortMergeJoin should not buffer all matching right side rows when bound condition is empty, this is unnecessary and can lead to performance degradation especially when spilling happens.

### Why are the changes needed?

Performance improvement.

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

No.

### How was this patch tested?

New UT.

Closes #29727 from peter-toth/SPARK-32730-improve-leftsemi-sortmergejoin-followup.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-11 13:42:33 +00:00
Wenchen Fan 9f4f49cbaa [SPARK-32853][SQL] Consecutive save/load calls in DataFrame/StreamReader/Writer should not fail
### What changes were proposed in this pull request?

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

In https://github.com/apache/spark/pull/29328 , we forbid the use case that path option and path parameter are both specified.  However,  it breaks some use cases:
```
val dfr =  spark.read.format(...).option(...)
dfr.load(path1).xxx
dfr.load(path2).xxx
```

The reason is that: `load` has side effects. It will set path option to the `DataFrameReader` instance. The next time you call `load`, Spark will fail because both path option and path parameter are specified.

This PR removes the side effect of `save`/`load`/`start`  to not set the path option.

### Why are the changes needed?

recover some use cases

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

Yes, some use cases fail before this PR, and can run successfully after this PR.

### How was this patch tested?

new tests

Closes #29723 from cloud-fan/df.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-11 06:15:58 -07:00
yangjiang fe2ab255d1 [MINOR][SQL] Fix a typo at 'spark.sql.sources.fileCompressionFactor' error message in SQLConf
### What changes were proposed in this pull request?

 fix typo in SQLConf

### Why are the changes needed?

typo fix to increase readability

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

### How was this patch tested?

no test

Closes #29668 from Ted-Jiang/fix_annotate.

Authored-by: yangjiang <yangjiang@ebay.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-11 08:05:34 -05:00
Wenchen Fan 328d81a2d1 [SPARK-32677][SQL][DOCS][MINOR] Improve code comment in CreateFunctionCommand
### What changes were proposed in this pull request?

We made a mistake in https://github.com/apache/spark/pull/29502, as there is no code comment to explain why we can't load the UDF class when creating functions. This PR improves the code comment.

### Why are the changes needed?

To avoid making the same mistake.

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

No

### How was this patch tested?

N/A

Closes #29713 from cloud-fan/comment.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-11 09:22:56 +09:00
Kousuke Saruta 5f468cc21e [SPARK-32822][SQL] Change the number of partitions to zero when a range is empty with WholeStageCodegen disabled or falled back
### What changes were proposed in this pull request?

This PR changes the behavior of RangeExec with WholeStageCodegen disabled or falled back to change the number of partitions to zero when a range is empty.

In the current master, if WholeStageCodegen effects, the number of partitions of an empty range will be changed to zero.
```
spark.range(1, 1, 1, 1000).rdd.getNumPartitions
res0: Int = 0
```
But it doesn't if WholeStageCodegen is disabled or falled back.
```
spark.conf.set("spark.sql.codegen.wholeStage", false)
spark.range(1, 1, 1, 1000).rdd.getNumPartitions
res2: Int = 1000
```

### Why are the changes needed?

To archive better performance even though WholeStageCodegen disabled or falled back.

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

Yes. the number of partitions gotten with `getNumPartitions` for an empty range will be changed when WholeStageCodegen is disabled.

### How was this patch tested?

New test.

Closes #29681 from sarutak/zero-size-range.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-11 09:11:35 +09:00
gengjiaan a22871f50a [SPARK-32777][SQL] Aggregation support aggregate function with multiple foldable expressions
### What changes were proposed in this pull request?
Spark SQL exists a bug show below:

```
spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2, 3)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 2, 3)|
+-----------------+--------------------+
|                1|                   1|
+-----------------+--------------------+

spark.sql(
  " SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)")
  .show()
+-----------------+--------------------+
|count(DISTINCT 2)|count(DISTINCT 3, 2)|
+-----------------+--------------------+
|                1|                   0|
+-----------------+--------------------+
```
The first query is correct, but the second query is not.
The root reason is the second query rewrited by `RewriteDistinctAggregates` who expand the output but lost the 2.

### Why are the changes needed?
Fix a bug.
`SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3, 2)` should return `1, 1`

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

### How was this patch tested?
New UT

Closes #29626 from beliefer/support-multiple-foldable-distinct-expressions.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:25:32 +00:00
Kent Yao 5669b212ec [SPARK-32840][SQL] Invalid interval value can happen to be just adhesive with the unit
### What changes were proposed in this pull request?
In this PR, we add a checker for STRING form interval value ahead for parsing multiple units intervals and fail directly if the interval value contains alphabets to prevent correctness issues like `interval '1 day 2' day`=`3 days`.

### Why are the changes needed?

fix correctness issue

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

yes, in spark 3.0.0 `interval '1 day 2' day`=`3 days` but now we fail with ParseException
### How was this patch tested?

add a test.

Closes #29708 from yaooqinn/SPARK-32840.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 11:20:05 +00:00
Takeshi Yamamuro 7eb76d6988 [SPARK-32828][SQL] Cast from a derived user-defined type to a base type
### What changes were proposed in this pull request?

This PR intends to fix an existing bug below in `UserDefinedTypeSuite`;
```
[info] - SPARK-19311: UDFs disregard UDT type hierarchy (931 milliseconds)
16:22:35.936 WARN org.apache.spark.sql.catalyst.expressions.SafeProjection: Expr codegen error and falling back to interpreter mode
org.apache.spark.SparkException: Cannot cast org.apache.spark.sql.ExampleSubTypeUDT46b1771f to org.apache.spark.sql.ExampleBaseTypeUDT31e8d979.
	at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeCastFunction(Cast.scala:891)
	at org.apache.spark.sql.catalyst.expressions.CastBase.doGenCode(Cast.scala:852)
	at org.apache.spark.sql.catalyst.expressions.Expression.$anonfun$genCode$3(Expression.scala:147)
    ...
```

### Why are the changes needed?

bugfix

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

No.

### How was this patch tested?

Added unit tests.

Closes #29691 from maropu/FixUdtBug.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-10 19:19:26 +09:00
Jungtaek Lim (HeartSaVioR) 8f61005723 [SPARK-32456][SS][FOLLOWUP] Update doc to note about using SQL statement with streaming Dataset
### What changes were proposed in this pull request?

This patch proposes to update the doc (both SS guide doc and Dataset dropDuplicates method doc) to leave a note to check on using SQL statements with streaming Dataset.

Once end users create a temp view based on streaming Dataset, they won't bother with thinking about "streaming" and do whatever they do with batch query. In many cases it works, but not just smoothly for the case when streaming aggregation is involved. They still need to concern about maintaining state store.

### Why are the changes needed?

Although SPARK-32456 fixed the weird error message, as a side effect some operations are enabled on streaming workload via SQL statement, which is error-prone if end users don't indicate what they're doing.

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

No.

### How was this patch tested?

Only doc change.

Closes #29461 from HeartSaVioR/SPARK-32456-FOLLOWUP-DOC.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 08:10:32 +00:00
Dongjoon Hyun 2f85f9516c [SPARK-32832][SS] Use CaseInsensitiveMap for DataStreamReader/Writer options
### What changes were proposed in this pull request?

This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following.
```scala
scala> spark.readStream.format("parquet").option("paTh", "1").option("PATH", "2").option("Path", "3").option("patH", "4").option("path", "5").load()
org.apache.spark.sql.AnalysisException: Path does not exist: 1;
```

### Why are the changes needed?

This will make the behavior deterministic.

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

Yes, but the previous behavior is indeterministic.

### How was this patch tested?

Pass the newly test cases.

Closes #29702 from dongjoon-hyun/SPARK-32832.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-09 23:41:32 -07:00
Jungtaek Lim (HeartSaVioR) db89b0e1b8 [SPARK-32831][SS] Refactor SupportsStreamingUpdate to represent actual meaning of the behavior
### What changes were proposed in this pull request?

This PR renames `SupportsStreamingUpdate` to `SupportsStreamingUpdateAsAppend` as the new interface name represents the actual behavior clearer. This PR also removes the `update()` method (so the interface is more likely a marker), as the implementations of `SupportsStreamingUpdateAsAppend` should support append mode by default, hence no need to trigger some flag on it.

### Why are the changes needed?

SupportsStreamingUpdate was intended to revive the functionality of Streaming update output mode for internal data sources, but despite the name, that interface isn't really used to do actual update on sink; all sinks are implementing this interface to do append, so strictly saying, it's just to support update as append. Renaming the interface would make it clear.

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

No, as the class is only for internal data sources.

### How was this patch tested?

Jenkins test will follow.

Closes #29693 from HeartSaVioR/SPARK-32831.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-09-10 15:33:18 +09:00
HyukjinKwon 4a096131ee Revert "[SPARK-32772][SQL][FOLLOWUP] Remove legacy silent support mode for spark-sql CLI"
This reverts commit f1f7ae420e.
2020-09-10 14:23:10 +09:00
Bryan Cutler e0538bd38c [SPARK-32312][SQL][PYTHON][TEST-JAVA11] Upgrade Apache Arrow to version 1.0.1
### What changes were proposed in this pull request?

Upgrade Apache Arrow to version 1.0.1 for the Java dependency and increase minimum version of PyArrow to 1.0.0.

This release marks a transition to binary stability of the columnar format (which was already informally backward-compatible going back to December 2017) and a transition to Semantic Versioning for the Arrow software libraries. Also note that the Java arrow-memory artifact has been split to separate dependence on netty-buffer and allow users to select an allocator. Spark will continue to use `arrow-memory-netty` to maintain performance benefits.

Version 1.0.0 - 1.0.0 include the following selected fixes/improvements relevant to Spark users:

ARROW-9300 - [Java] Separate Netty Memory to its own module
ARROW-9272 - [C++][Python] Reduce complexity in python to arrow conversion
ARROW-9016 - [Java] Remove direct references to Netty/Unsafe Allocators
ARROW-8664 - [Java] Add skip null check to all Vector types
ARROW-8485 - [Integration][Java] Implement extension types integration
ARROW-8434 - [C++] Ipc RecordBatchFileReader deserializes the Schema multiple times
ARROW-8314 - [Python] Provide a method to select a subset of columns of a Table
ARROW-8230 - [Java] Move Netty memory manager into a separate module
ARROW-8229 - [Java] Move ArrowBuf into the Arrow package
ARROW-7955 - [Java] Support large buffer for file/stream IPC
ARROW-7831 - [Java] unnecessary buffer allocation when calling splitAndTransferTo on variable width vectors
ARROW-6111 - [Java] Support LargeVarChar and LargeBinary types and add integration test with C++
ARROW-6110 - [Java] Support LargeList Type and add integration test with C++
ARROW-5760 - [C++] Optimize Take implementation
ARROW-300 - [Format] Add body buffer compression option to IPC message protocol using LZ4 or ZSTD
ARROW-9098 - RecordBatch::ToStructArray cannot handle record batches with 0 column
ARROW-9066 - [Python] Raise correct error in isnull()
ARROW-9223 - [Python] Fix to_pandas() export for timestamps within structs
ARROW-9195 - [Java] Wrong usage of Unsafe.get from bytearray in ByteFunctionsHelper class
ARROW-7610 - [Java] Finish support for 64 bit int allocations
ARROW-8115 - [Python] Conversion when mixing NaT and datetime objects not working
ARROW-8392 - [Java] Fix overflow related corner cases for vector value comparison
ARROW-8537 - [C++] Performance regression from ARROW-8523
ARROW-8803 - [Java] Row count should be set before loading buffers in VectorLoader
ARROW-8911 - [C++] Slicing a ChunkedArray with zero chunks segfaults

View release notes here:
https://arrow.apache.org/release/1.0.1.html
https://arrow.apache.org/release/1.0.0.html

### Why are the changes needed?

Upgrade brings fixes, improvements and stability guarantees.

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

No

### How was this patch tested?

Existing tests with pyarrow 1.0.0 and 1.0.1

Closes #29686 from BryanCutler/arrow-upgrade-100-SPARK-32312.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-10 14:16:19 +09:00
Kent Yao 9ab8a2c36d [SPARK-32826][SQL] Set the right column size for the null type in SparkGetColumnsOperation
### What changes were proposed in this pull request?

In Spark 3.0.0, the SparkGetColumnsOperation can not recognize NULL columns but now we can because the side effect of https://issues.apache.org/jira/browse/SPARK-32696 / f14f3742e0, but the test coverage for this change was not added.

In Spark, the column size for null fields should be 1, in this PR, we set the right column size for the null type.

### Why are the changes needed?

test coverage and fix the client-side information about the null type through jdbc

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

NO
### How was this patch tested?

added ut both for this pr and SPARK-32696

Closes #29687 from yaooqinn/SPARK-32826.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 04:53:38 +00:00
Liang-Chi Hsieh add267c4de [SPARK-32819][SQL] ignoreNullability parameter should be effective recursively
### What changes were proposed in this pull request?

This patch proposes to check `ignoreNullability` parameter recursively in `equalsStructurally` method.

### Why are the changes needed?

`equalsStructurally` is used to check type equality. We can optionally ask to ignore nullability check. But the parameter `ignoreNullability` is not passed recursively down to nested types. So it produces weird error like:

```
data type mismatch: argument 3 requires array<array<string>> type, however ... is of array<array<string>> type.
```

when running the query `select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array( x ) ) )`.

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

Yes, fixed a bug when running user query.

### How was this patch tested?

Unit tests.

Closes #29698 from viirya/SPARK-32819.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 02:53:22 +00:00
Dongjoon Hyun 06a994517f [SPARK-32836][SS][TESTS] Fix DataStreamReaderWriterSuite to check writer options correctly
### What changes were proposed in this pull request?

This PR aims to fix the test coverage at `DataStreamReaderWriterSuite`.

### Why are the changes needed?

Currently, the test case checks `DataStreamReader` options instead of `DataStreamWriter` options.

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

No.

### How was this patch tested?

Pass the revised test case.

Closes #29701 from dongjoon-hyun/SPARK-32836.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-09 19:46:55 -07:00
Terry Kim ab2fa881ed [SPARK-32516][SQL][FOLLOWUP] Remove unnecessary check if path string is empty for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start()
### What changes were proposed in this pull request?

This PR is a follow up to https://github.com/apache/spark/pull/29543#discussion_r485409606, which correctly points out that the check for the empty string is not necessary.

### Why are the changes needed?

The unnecessary check actually could cause more confusion.

For example,
```scala
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("")
java.lang.IllegalArgumentException: Can not create a Path from an empty string
  at org.apache.hadoop.fs.Path.checkPathArg(Path.java:168)
```
even when `path` option is available. This PR addresses to fix this confusion.

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

Yes, now the above example prints the consistent exception message whether the path parameter value is empty or not.
```scala
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("")
org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.;
  at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:290)
  at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:856)
  ... 47 elided
```

### How was this patch tested?

Added unit tests.

Closes #29697 from imback82/SPARK-32516-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-10 01:48:51 +00:00
Wenchen Fan f7995c576a Revert "[SPARK-32677][SQL] Load function resource before create"
This reverts commit 05fcf26b79.
2020-09-09 18:15:22 +00:00
Tathagata Das e4237bbda6 [SPARK-32794][SS] Fixed rare corner case error in micro-batch engine with some stateful queries + no-data-batches + V1 sources
### What changes were proposed in this pull request?
Make MicroBatchExecution explicitly call `getBatch` when the start and end offsets are the same.

### Why are the changes needed?

Structured Streaming micro-batch engine has the contract with V1 data sources that, after a restart, it will call `source.getBatch()` on the last batch attempted before the restart. However, a very rare combination of sequences violates this contract. It occurs only when
- The streaming query has specific types of stateful operations with watermarks (e.g., aggregation in append, mapGroupsWithState with timeouts).
    - These queries can execute a batch even without new data when the previous updates the watermark and the stateful ops are such that the new watermark can cause new output/cleanup. Such batches are called no-data-batches.
- The last batch before termination was an incomplete no-data-batch. Upon restart, the micro-batch engine fails to call `source.getBatch` when attempting to re-execute the incomplete no-data-batch.

This occurs because no-data-batches has the same and end offsets, and when a batch is executed, if the start and end offset is same then calling `source.getBatch` is skipped as it is assumed the generated plan will be empty. This only affects V1 data sources like Delta and Autoloader which rely on this invariant to detect in the source whether the query is being started from scratch or restarted.

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

### How was this patch tested?

New unit test with a mock v1 source that fails without the fix.

Closes #29651 from tdas/SPARK-32794.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2020-09-09 13:35:51 -04:00
yangjie01 fc10511d15 [SPARK-32755][SQL][FOLLOWUP] Ensure -- method of AttributeSet have same behavior under Scala 2.12 and 2.13
### What changes were proposed in this pull request?
 `--` method of `AttributeSet` behave differently under Scala 2.12 and 2.13 because `--` method of `LinkedHashSet` in Scala 2.13 can't maintains the insertion order.

This pr use a Scala 2.12 based code to ensure `--` method of AttributeSet have same behavior under Scala 2.12 and 2.13.

### Why are the changes needed?
The behavior of `AttributeSet`  needs to be compatible with Scala 2.12 and 2.13

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

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

Scala 2.13: Manual test sub-suites of `PlanStabilitySuite`

- **Before** :293 TESTS FAILED

- **After**:13 TESTS FAILED(The remaining failures are not associated with the current issue)

Closes #29689 from LuciferYang/SPARK-32755-FOLLOWUP.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-09 14:40:49 +00:00
yangjie01 513d51a2c5 [SPARK-32808][SQL] Fix some test cases of sql/core module in scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32808](https://issues.apache.org/jira/browse/SPARK-32808), total of 26 failed test cases were fixed, the related suite as follow:

- `StreamingAggregationSuite` related test cases (2 FAILED -> Pass)

- `GeneratorFunctionSuite` related test cases (2 FAILED -> Pass)

- `UDFSuite` related test cases (2 FAILED -> Pass)

- `SQLQueryTestSuite` related test cases (5 FAILED -> Pass)

- `WholeStageCodegenSuite` related test cases (1 FAILED -> Pass)

- `DataFrameSuite` related test cases (3 FAILED -> Pass)

- `OrcV1QuerySuite\OrcV2QuerySuite` related test cases (4 FAILED -> Pass)

- `ExpressionsSchemaSuite` related test cases (1 FAILED -> Pass)

- `DataFrameStatSuite` related test cases (1 FAILED -> Pass)

- `JsonV1Suite\JsonV2Suite\JsonLegacyTimeParserSuite` related test cases (6 FAILED -> Pass)

The main change of this pr as following:

- Fix Scala 2.13 compilation problems in   `ShuffleBlockFetcherIterator`  and `Analyzer`

- Specified `Seq` to `scala.collection.Seq` in `objects.scala` and `GenericArrayData` because internal use `Seq` maybe `mutable.ArraySeq` and not easy to call `.toSeq`

- Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but `Seq` is `immutable.Seq` in Scala 2.13

- Use a compatible way to let `+` and `-` method  of `Decimal` having the same behavior in Scala 2.12 and Scala 2.13

- Call `toList` in `RelationalGroupedDataset.toDF` method when `groupingExprs` is `Stream` type because `Stream` can't serialize in Scala 2.13

- Add a manual sort to `classFunsMap` in `ExpressionsSchemaSuite` because `Iterable.groupBy` in Scala 2.13 has different result with `TraversableLike.groupBy`  in Scala 2.12

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

Should specified `Seq` to `scala.collection.Seq`  when we call `Row.getAs[Seq]` and `Row.get(i).asInstanceOf[Seq]` because the data maybe `mutable.ArraySeq` but the `Seq` is `immutable.Seq` in Scala 2.13

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/core -Pscala-2.13 -am
mvn test -pl sql/core -Pscala-2.13
```

**Before**
```
Tests: succeeded 8166, failed 319, canceled 1, ignored 52, pending 0
*** 319 TESTS FAILED ***

```

**After**

```
Tests: succeeded 8204, failed 286, canceled 1, ignored 52, pending 0
*** 286 TESTS FAILED ***

```

Closes #29660 from LuciferYang/SPARK-32808.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-09 08:53:44 -05:00
Liang-Chi Hsieh de0dc52a84 [SPARK-32813][SQL] Get default config of ParquetSource vectorized reader if no active SparkSession
### What changes were proposed in this pull request?

If no active SparkSession is available, let `FileSourceScanExec.needsUnsafeRowConversion` look at default SQL config of ParquetSource vectorized reader instead of failing the query execution.

### Why are the changes needed?

Fix a bug that if no active SparkSession is available, file-based data source scan for Parquet Source will throw exception.

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

Yes, this change fixes the bug.

### How was this patch tested?

Unit test.

Closes #29667 from viirya/SPARK-32813.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-09 12:23:05 +09:00
Max Gekk adc8d687ce [SPARK-32810][SQL][TESTS][FOLLOWUP] Check path globbing in JSON/CSV datasources v1 and v2
### What changes were proposed in this pull request?
In the PR, I propose to move the test `SPARK-32810: CSV and JSON data sources should be able to read files with escaped glob metacharacter in the paths` from `DataFrameReaderWriterSuite` to `CSVSuite` and to `JsonSuite`. This will allow to run the same test in `CSVv1Suite`/`CSVv2Suite` and in `JsonV1Suite`/`JsonV2Suite`.

### Why are the changes needed?
To improve test coverage by checking JSON/CSV datasources v1 and v2.

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

### How was this patch tested?
By running affected test suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.execution.datasources.csv.*"
$ build/sbt "sql/test:testOnly org.apache.spark.sql.execution.datasources.json.*"
```

Closes #29684 from MaxGekk/globbing-paths-when-inferring-schema-dsv2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-09 10:29:58 +09:00
manuzhang 96ff87dce8 [SPARK-32753][SQL][FOLLOWUP] Fix indentation and clean up view in test
### What changes were proposed in this pull request?
Fix indentation and clean up view in the test added by https://github.com/apache/spark/pull/29593.

### Why are the changes needed?
Address review comments in https://github.com/apache/spark/pull/29665.

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

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

Closes #29682 from manuzhang/spark-32753-followup.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-09 10:20:21 +09:00
Zhenhua Wang e7d9a24565 [SPARK-32817][SQL] DPP throws error when broadcast side is empty
### What changes were proposed in this pull request?

In `SubqueryBroadcastExec.relationFuture`, if the `broadcastRelation` is an `EmptyHashedRelation`, then `broadcastRelation.keys()` will throw `UnsupportedOperationException`.

### Why are the changes needed?

To fix a bug.

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

No.

### How was this patch tested?

Added a new test.

Closes #29671 from wzhfy/dpp_empty_broadcast.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-08 21:36:21 +09:00
sychen bd3dc2f54d [SPARK-31511][FOLLOW-UP][TEST][SQL] Make BytesToBytesMap iterators thread-safe
### What changes were proposed in this pull request?
Before SPARK-31511 is fixed, `BytesToBytesMap` iterator() is not thread-safe and may cause data inaccuracy.
We need to add a unit test.

### Why are the changes needed?
Increase test coverage to ensure that iterator() is thread-safe.

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

### How was this patch tested?
add ut

Closes #29669 from cxzl25/SPARK-31511-test.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-08 11:54:04 +00:00
Zhenhua Wang 55d38a479b [SPARK-32748][SQL] Revert "Support local property propagation in SubqueryBroadcastExec"
### What changes were proposed in this pull request?

This reverts commit 04f7f6dac0 due to the discussion in [comment](https://github.com/apache/spark/pull/29589#discussion_r484657207).

### Why are the changes needed?

Based on  the discussion in [comment](https://github.com/apache/spark/pull/29589#discussion_r484657207), propagation for thread local properties in `SubqueryBroadcastExec` is not necessary, since they will be propagated by broadcast exchange threads anyway.

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

No.

### How was this patch tested?

Also revert the added test.

Closes #29674 from wzhfy/revert_dpp_thread_local.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-08 20:20:16 +09:00
Wenchen Fan 4144b6da52 [SPARK-32764][SQL] -0.0 should be equal to 0.0
### What changes were proposed in this pull request?

This is a Spark 3.0 regression introduced by https://github.com/apache/spark/pull/26761. We missed a corner case that `java.lang.Double.compare` treats 0.0 and -0.0 as different, which breaks SQL semantic.

This PR adds back the `OrderingUtil`, to provide custom compare methods that take care of 0.0 vs -0.0

### Why are the changes needed?

Fix a correctness bug.

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

Yes, now `SELECT  0.0 > -0.0` returns false correctly as Spark 2.x.

### How was this patch tested?

new tests

Closes #29647 from cloud-fan/float.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-07 20:43:43 -07:00
Wenchen Fan 117a6f135b [SPARK-32638][SQL][FOLLOWUP] Move the plan rewriting methods to QueryPlan
### What changes were proposed in this pull request?

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

It moves the plan rewriting methods from `Analyzer` to `QueryPlan`, so that it can work with `SparkPlan` as well. This PR also does an improvement to support a corner case (The attribute to be replace stays together with an unresolved attribute), and make it more general, so that `WidenSetOperationTypes` can rewrite the plan in one shot like before.

### Why are the changes needed?

Code cleanup and generalize.

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

no

### How was this patch tested?

existing test

Closes #29643 from cloud-fan/cleanup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-08 09:54:05 +09:00
Max Gekk 954cd9feaa [SPARK-32810][SQL] CSV/JSON data sources should avoid globbing paths when inferring schema
### What changes were proposed in this pull request?
In the PR, I propose to fix an issue with the CSV and JSON data sources in Spark SQL when both of the following are true:
* no user specified schema
* some file paths contain escaped glob metacharacters, such as `[``]`, `{``}`, `*` etc.

### Why are the changes needed?
To fix the issue when the follow two queries try to read from paths `[abc].csv` and `[abc].json`:
```scala
spark.read.csv("""/tmp/\[abc\].csv""").show
spark.read.json("""/tmp/\[abc\].json""").show
```
but would end up hitting an exception:
```
org.apache.spark.sql.AnalysisException: Path does not exist: file:/tmp/[abc].csv;
  at org.apache.spark.sql.execution.datasources.DataSource$.$anonfun$checkAndGlobPathIfNecessary$1(DataSource.scala:722)
  at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:244)
  at scala.collection.immutable.List.foreach(List.scala:392)
```

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

### How was this patch tested?
Added new test cases in `DataFrameReaderWriterSuite`.

Closes #29659 from MaxGekk/globbing-paths-when-inferring-schema.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-08 09:42:59 +09:00
manuzhang c43460cf82 [SPARK-32753][SQL] Only copy tags to node with no tags
### What changes were proposed in this pull request?
Only copy tags to node with no tags when transforming plans.

### Why are the changes needed?
cloud-fan [made a good point](https://github.com/apache/spark/pull/29593#discussion_r482013121) that it doesn't make sense to append tags to existing nodes when nodes are removed. That will cause such bugs as duplicate rows when deduplicating and repartitioning by the same column with AQE.

```
spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1")
val df = spark.sql("select id from v1 group by id distribute by id")
println(df.collect().toArray.mkString(","))
println(df.queryExecution.executedPlan)

// With AQE
[4],[0],[3],[2],[1],[7],[6],[8],[5],[9],[4],[0],[3],[2],[1],[7],[6],[8],[5],[9]
AdaptiveSparkPlan(isFinalPlan=true)
+- CustomShuffleReader local
   +- ShuffleQueryStage 0
      +- Exchange hashpartitioning(id#183L, 10), true
         +- *(3) HashAggregate(keys=[id#183L], functions=[], output=[id#183L])
            +- Union
               :- *(1) Range (0, 10, step=1, splits=2)
               +- *(2) Range (0, 10, step=1, splits=2)

// Without AQE
[4],[7],[0],[6],[8],[3],[2],[5],[1],[9]
*(4) HashAggregate(keys=[id#206L], functions=[], output=[id#206L])
+- Exchange hashpartitioning(id#206L, 10), true
   +- *(3) HashAggregate(keys=[id#206L], functions=[], output=[id#206L])
      +- Union
         :- *(1) Range (0, 10, step=1, splits=2)
         +- *(2) Range (0, 10, step=1, splits=2)
```

It's too expensive to detect node removal so we make a compromise only to copy tags to node with no tags.

### Does this PR introduce _any_ user-facing change?
Yes. Fix a bug.

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

Closes #29593 from manuzhang/spark-32753.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 16:08:57 +00:00
Zhenhua Wang 04f7f6dac0 [SPARK-32748][SQL] Support local property propagation in SubqueryBroadcastExec
### What changes were proposed in this pull request?

Since [SPARK-22590](2854091d12), local property propagation is supported through `SQLExecution.withThreadLocalCaptured` in both `BroadcastExchangeExec` and `SubqueryExec` when computing `relationFuture`. This pr adds the support in `SubqueryBroadcastExec`.

### Why are the changes needed?

Local property propagation is missed in `SubqueryBroadcastExec`.

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

No.

### How was this patch tested?

Add a new test.

Closes #29589 from wzhfy/thread_local.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 06:26:14 +00:00
sandeep.katta b0322bf05a [SPARK-32779][SQL] Avoid using synchronized API of SessionCatalog in withClient flow, this leads to DeadLock
### What changes were proposed in this pull request?

No need of using database name in `loadPartition` API of `Shim_v3_0` to get the hive table, in hive there is a overloaded method which gives hive table using table name. By using this API dependency with `SessionCatalog` can be removed in Shim layer

### Why are the changes needed?
To avoid deadlock when communicating with Hive metastore 3.1.x
```
Found one Java-level deadlock:
=============================
"worker3":
  waiting to lock monitor 0x00007faf0be602b8 (object 0x00000007858f85f0, a org.apache.spark.sql.hive.HiveSessionCatalog),
  which is held by "worker0"
"worker0":
  waiting to lock monitor 0x00007faf0be5fc88 (object 0x0000000785c15c80, a org.apache.spark.sql.hive.HiveExternalCatalog),
  which is held by "worker3"

Java stack information for the threads listed above:
===================================================
"worker3":
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.getCurrentDatabase(SessionCatalog.scala:256)
  - waiting to lock <0x00000007858f85f0> (a org.apache.spark.sql.hive.HiveSessionCatalog)
  at org.apache.spark.sql.hive.client.Shim_v3_0.loadPartition(HiveShim.scala:1332)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$loadPartition$1(HiveClientImpl.scala:870)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$Lambda$4459/1387095575.apply$mcV$sp(Unknown Source)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
  at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
  at org.apache.spark.sql.hive.client.HiveClientImpl$$Lambda$2227/313239499.apply(Unknown Source)
  at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
  at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
  - locked <0x0000000785ef9d78> (a org.apache.spark.sql.hive.client.IsolatedClientLoader)
  at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
  at org.apache.spark.sql.hive.client.HiveClientImpl.loadPartition(HiveClientImpl.scala:860)
  at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$loadPartition$1(HiveExternalCatalog.scala:911)
  at org.apache.spark.sql.hive.HiveExternalCatalog$$Lambda$4457/2037578495.apply$mcV$sp(Unknown Source)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
  at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
  - locked <0x0000000785c15c80> (a org.apache.spark.sql.hive.HiveExternalCatalog)
  at org.apache.spark.sql.hive.HiveExternalCatalog.loadPartition(HiveExternalCatalog.scala:890)
  at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.loadPartition(ExternalCatalogWithListener.scala:179)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.loadPartition(SessionCatalog.scala:512)
  at org.apache.spark.sql.execution.command.LoadDataCommand.run(tables.scala:383)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  - locked <0x00000007b1690ff8> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$$Lambda$2084/428667685.apply(Unknown Source)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
  at org.apache.spark.sql.Dataset$$Lambda$2085/559530590.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2093/139449177.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2086/1088974677.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.Dataset$$$Lambda$1959/1977822284.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:606)
  at org.apache.spark.sql.SparkSession$$Lambda$1899/424830920.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
  at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anon$1.run(<console>:45)
  at java.lang.Thread.run(Thread.java:748)
"worker0":
  at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
  - waiting to lock <0x0000000785c15c80
  > (a org.apache.spark.sql.hive.HiveExternalCatalog)
  at org.apache.spark.sql.hive.HiveExternalCatalog.tableExists(HiveExternalCatalog.scala:851)
  at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.tableExists(ExternalCatalogWithListener.scala:146)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.tableExists(SessionCatalog.scala:432)
  - locked <0x00000007858f85f0> (a org.apache.spark.sql.hive.HiveSessionCatalog)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.requireTableExists(SessionCatalog.scala:185)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.loadPartition(SessionCatalog.scala:509)
  at org.apache.spark.sql.execution.command.LoadDataCommand.run(tables.scala:383)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  - locked <0x00000007b529af58> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$$Lambda$2084/428667685.apply(Unknown Source)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3616)
  at org.apache.spark.sql.Dataset$$Lambda$2085/559530590.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:100)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2093/139449177.apply(Unknown Source)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:87)
  at org.apache.spark.sql.execution.SQLExecution$$$Lambda$2086/1088974677.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3614)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.Dataset$$$Lambda$1959/1977822284.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:606)
  at org.apache.spark.sql.SparkSession$$Lambda$1899/424830920.apply(Unknown Source)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
  at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anon$1.run(<console>:45)
  at java.lang.Thread.run(Thread.java:748)

Found 1 deadlock.
```

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

### How was this patch tested?
Tested using below script by executing in spark-shell and I found no dead lock

launch spark-shell using ./bin/spark-shell --conf "spark.sql.hive.metastore.jars=maven" --conf spark.sql.hive.metastore.version=3.1 --conf spark.hadoop.datanucleus.schema.autoCreateAll=true

**code**
```
def testHiveDeadLock = {
      import scala.collection.mutable.ArrayBuffer
      import scala.util.Random
      println("test hive DeadLock")
      spark.sql("drop database if exists testDeadLock cascade")
      spark.sql("create database testDeadLock")
      spark.sql("use testDeadLock")
      val tableCount = 100
      val tableNamePrefix = "testdeadlock"
      for (i <- 0 until tableCount) {
        val tableName = s"$tableNamePrefix${i + 1}"
        spark.sql(s"drop table if exists $tableName")
        spark.sql(s"create table $tableName (a bigint) partitioned by (b bigint) stored as orc")
      }

      val threads = new ArrayBuffer[Thread]
      for (i <- 0 until tableCount) {
        threads.append(new Thread( new Runnable {
          override def run: Unit = {
            val tableName = s"$tableNamePrefix${i + 1}"
            val rand = Random
            val df = spark.range(0, 20000).toDF("a")
            val location = s"/tmp/${rand.nextLong.abs}"
            df.write.mode("overwrite").orc(location)
            spark.sql(
              s"""
        LOAD DATA LOCAL INPATH '$location' INTO TABLE $tableName partition (b=$i)""")
          }
        }, s"worker$i"))
        threads(i).start()
      }

      for (i <- 0 until tableCount) {
        println(s"Joining with thread $i")
        threads(i).join()
      }
      for (i <- 0 until tableCount) {
        val tableName = s"$tableNamePrefix${i + 1}"
        spark.sql(s"select count(*) from $tableName").show(false)
      }
      println("All done")
    }

    for(i <- 0 until 100) {
      testHiveDeadLock
      println(s"completed {$i}th iteration")
    }
  }
```

Closes #29649 from sandeep-katta/metastore3.1DeadLock.

Authored-by: sandeep.katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-07 15:10:33 +09:00
ulysses 05fcf26b79 [SPARK-32677][SQL] Load function resource before create
### What changes were proposed in this pull request?

Change `CreateFunctionCommand` code that add class check before create function.

### Why are the changes needed?

We have different behavior between create permanent function and temporary function when function class is invaild. e.g.,
```
create function f as 'test.non.exists.udf';
-- Time taken: 0.104 seconds

create temporary function f as 'test.non.exists.udf'
-- Error in query: Can not load class 'test.non.exists.udf' when registering the function 'f', please make sure it is on the classpath;
```

And Hive also fails both of them.

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

Yes, user will get exception when create a invalid udf.

### How was this patch tested?

New test.

Closes #29502 from ulysses-you/function.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 06:00:23 +00:00
Kent Yao de44e9cfa0 [SPARK-32785][SQL] Interval with dangling parts should not results null
### What changes were proposed in this pull request?

bugfix for incomplete interval values, e.g. interval '1', interval '1 day 2', currently these cases will result null, but actually we should fail them with IllegalArgumentsException

### Why are the changes needed?

correctness

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

yes, incomplete intervals will throw exception now

#### before
```
bin/spark-sql -S -e "select interval '1', interval '+', interval '1 day -'"

NULL NULL NULL
```
#### after

```
-- !query
select interval '1'
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

Cannot parse the INTERVAL value: 1(line 1, pos 7)

== SQL ==
select interval '1'
```

### How was this patch tested?

unit tests added

Closes #29635 from yaooqinn/SPARK-32785.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-07 05:11:30 +00:00
Eren Avsarogullari f5360e761e [SPARK-32548][SQL] - Add Application attemptId support to SQL Rest API
### What changes were proposed in this pull request?
Currently, Spark Public Rest APIs support Application attemptId except SQL API. This causes `no such app: application_X` issue when the application has `attemptId` (e.g: YARN cluster mode).

Please find existing and supported Rest endpoints with attemptId.
```
// Existing Rest Endpoints
applications/{appId}/sql
applications/{appId}/sql/{executionId}

// Rest Endpoints required support
applications/{appId}/{attemptId}/sql
applications/{appId}/{attemptId}/sql/{executionId}
```
Also fixing following compile warning on `SqlResourceSuite`:
```
[WARNING] [Warn] ~/spark/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceSuite.scala:67: Reference to uninitialized value edges
```
### Why are the changes needed?
This causes `no such app: application_X` issue when the application has `attemptId`.

### Does this PR introduce _any_ user-facing change?
Not yet because SQL Rest API is being planned to release with `Spark 3.1`.

### How was this patch tested?
1. New Unit tests are added for existing Rest endpoints. `attemptId` seems not coming in `local-mode` and coming in `YARN cluster mode` so could not be added for `attemptId` case (Suggestions are welcome).
2. Also, patch has been tested manually through both Spark Core and History Server Rest APIs.

Closes #29364 from erenavsarogullari/SPARK-32548.

Authored-by: Eren Avsarogullari <erenavsarogullari@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-09-06 19:23:12 +08:00
Ali Afroozeh f55694638d [SPARK-32800][SQL] Remove ExpressionSet from the 2.13 branch
### What changes were proposed in this pull request?
This PR is a followup on #29598 and removes the `ExpressionSet` class from the 2.13 branch.

### Why are the changes needed?
`ExpressionSet` does not extend Scala `Set` anymore and this class is no longer needed in the 2.13 branch.

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

### How was this patch tested?
Passes existing tests

Closes #29648 from dbaliafroozeh/RemoveExpressionSetFrom2.13Branch.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-06 09:44:07 +09:00
Yuming Wang 0b3bb45b89 [SPARK-32791][SQL] Non-partitioned table metric should not have dynamic partition pruning time
### What changes were proposed in this pull request?

This pr make non-partitioned table metric should not have dynamic partition pruning time.

### Why are the changes needed?

It is useless for non-partitioned table.

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

No.

### How was this patch tested?

Manual test

Before this pr:
![image](https://user-images.githubusercontent.com/5399861/92141803-87fed380-ee45-11ea-9784-09625b246fea.png)
After this pr:
![image](https://user-images.githubusercontent.com/5399861/92141774-7c131180-ee45-11ea-8a9e-6775c592f496.png)

Closes #29641 from wangyum/SPARK-32791.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-05 23:49:17 +08:00
yangjie 1de272f98d [SPARK-32762][SQL][TEST] Enhance the verification of ExpressionsSchemaSuite to sql-expression-schema.md
### What changes were proposed in this pull request?
`sql-expression-schema.md` automatically generated by `ExpressionsSchemaSuite`, but only expressions entries are checked in `ExpressionsSchemaSuite`. So if we manually modify the contents of the file,  `ExpressionsSchemaSuite` does not necessarily guarantee the correctness of the it some times. For example, [Spark-24884](https://github.com/apache/spark/pull/27507) added `regexp_extract_all`  expression support, and manually modify the `sql-expression-schema.md` but not change the content of `Number of queries` cause file content inconsistency.

Some additional checks have been added to `ExpressionsSchemaSuite` to improve the correctness guarantee of `sql-expression-schema.md` as follow:

- `Number of queries` should equals size of `expressions entries` in `sql-expression-schema.md`

- `Number of expressions that missing example` should equals size of `Expressions missing examples` in `sql-expression-schema.md`

- `MissExamples` from case should same as  `expectedMissingExamples` from `sql-expression-schema.md`

### Why are the changes needed?
Ensure the correctness of `sql-expression-schema.md` content.

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

### How was this patch tested?
Enhanced ExpressionsSchemaSuite

Closes #29608 from LuciferYang/sql-expression-schema.

Authored-by: yangjie <yangjie@MacintoshdeMacBook-Pro.local>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-04 09:40:35 +09:00
Yuming Wang f1f7ae420e [SPARK-32772][SQL][FOLLOWUP] Remove legacy silent support mode for spark-sql CLI
### What changes were proposed in this pull request?

Remove legacy silent support mode for spark-sql CLI.

### Why are the changes needed?

https://github.com/apache/spark/pull/29619 add new silent mode. We can remove legacy silent support mode.

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

No.

### How was this patch tested?

Manual test:
```
spark-sql> LM-SHC-16508156:spark yumwang$ bin/spark-sql -S
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
20/09/03 09:06:12 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/09/03 09:06:16 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/09/03 09:06:16 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
20/09/03 09:06:19 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
20/09/03 09:06:19 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore yumwang10.226.196.190
spark-sql> select * from test1;
1
spark-sql> select * from test1;
1

```

Closes #29631 from wangyum/SPARK-32772.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-04 08:38:35 +08:00
Zhenhua Wang e693df2a07 [SPARK-32786][SQL][TEST] Improve performance for some slow DPP tests
### What changes were proposed in this pull request?

The whole `DynamicPartitionPruningSuite` takes about 2 min on my laptop (either AE on or off). The slowest tests are `test("simple inner join triggers DPP with mock-up tables")` and `test("cleanup any DPP filter that isn't pushed down due to expression id clashes")`, which totally take about 1 min.

We can reuse existing test tables or use smaller tables to reduce the cost. After that, the two tests takes only about 1 sec in total, leading to 2x speedup for the suite.

### Why are the changes needed?

To speedup DPP test suites.

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

No.

### How was this patch tested?

Modified two existing tests.

Closes #29636 from wzhfy/improve_dpp_test.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-09-04 09:33:20 +09:00
Wenchen Fan 76330e0295 [SPARK-32788][SQL] non-partitioned table scan should not have partition filter
### What changes were proposed in this pull request?

This PR fixes a bug `FileSourceStrategy`, which generates partition filters even if the table is not partitioned. This can confuse `FileSourceScanExec`, which mistakenly think the table is partitioned and tries to update the `numPartitions` metrics, and cause a failure. We should not generate partition filters for non-partitioned table.

### Why are the changes needed?

The bug was exposed by https://github.com/apache/spark/pull/29436.

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

Yes, fix a bug.

### How was this patch tested?

new test

Closes #29637 from cloud-fan/refactor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-03 23:49:17 +08:00
Takeshi Yamamuro a6114d8fb8 [SPARK-32638][SQL] Corrects references when adding aliases in WidenSetOperationTypes
### What changes were proposed in this pull request?

This PR intends to fix a bug where references can be missing when adding aliases to widen data types in `WidenSetOperationTypes`. For example,
```
CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v);
SELECT t.v FROM (
  SELECT v FROM t3
  UNION ALL
  SELECT v + v AS v FROM t3
) t;

org.apache.spark.sql.AnalysisException: Resolved attribute(s) v#1 missing from v#3 in operator !Project [v#1]. Attribute(s) with the same name appear in the operation: v. Please check if the right attribute(s) are used.;;
!Project [v#1]  <------ the reference got missing
+- SubqueryAlias t
   +- Union
      :- Project [cast(v#1 as decimal(11,0)) AS v#3]
      :  +- Project [v#1]
      :     +- SubqueryAlias t3
      :        +- SubqueryAlias tbl
      :           +- LocalRelation [v#1]
      +- Project [v#2]
         +- Project [CheckOverflow((promote_precision(cast(v#1 as decimal(11,0))) + promote_precision(cast(v#1 as decimal(11,0)))), DecimalType(11,0), true) AS v#2]
            +- SubqueryAlias t3
               +- SubqueryAlias tbl
                  +- LocalRelation [v#1]
```
In the case, `WidenSetOperationTypes` added the alias `cast(v#1 as decimal(11,0)) AS v#3`, then the reference in the top `Project` got missing. This PR correct the reference (`exprId` and widen `dataType`) after adding aliases in the rule.

### Why are the changes needed?

bugfixes

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

No

### How was this patch tested?

Added unit tests

Closes #29485 from maropu/SPARK-32638.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 14:48:26 +00:00
Peter Toth ffd5227543 [SPARK-32730][SQL] Improve LeftSemi and Existence SortMergeJoin right side buffering
### What changes were proposed in this pull request?

LeftSemi and Existence SortMergeJoin should not buffer all matching right side rows when bound condition is empty, this is unnecessary and can lead to performance degradation especially when spilling happens.

### Why are the changes needed?

Performance improvement.

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

No.

### How was this patch tested?

New UT and TPCDS benchmarks.

Closes #29572 from peter-toth/SPARK-32730-improve-leftsemi-sortmergejoin.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 14:17:34 +00:00
Ali Afroozeh 0a6043f683 [SPARK-32755][SQL] Maintain the order of expressions in AttributeSet and ExpressionSet
### What changes were proposed in this pull request?
This PR changes `AttributeSet` and `ExpressionSet` to maintain the insertion order of the elements. More specifically, we:
- change the underlying data structure of `AttributeSet` from `HashSet` to `LinkedHashSet` to maintain the insertion order.
- `ExpressionSet` already uses a list to keep track of the expressions, however, since it is extending Scala's immutable.Set class, operations such as map and flatMap are delegated to the immutable.Set itself. This means that the result of these operations is not an instance of ExpressionSet anymore, rather it's a implementation picked up by the parent class. We also remove this inheritance from `immutable.Set `and implement the needed methods directly. ExpressionSet has a very specific semantics and it does not make sense to extend `immutable.Set` anyway.
- change the `PlanStabilitySuite` to not sort the attributes, to be able to catch changes in the order of expressions in different runs.

### Why are the changes needed?
Expressions identity is based on the `ExprId` which is an auto-incremented number. This means that the same query can yield a query plan with different expression ids in different runs. `AttributeSet` and `ExpressionSet` internally use a `HashSet` as the underlying data structure, and therefore cannot guarantee the a fixed order of operations in different runs. This can be problematic in cases we like to check for plan changes in different runs.

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

### How was this patch tested?
Passes `PlanStabilitySuite` after regenerating the golden files.

Closes #29598 from dbaliafroozeh/FixOrderOfExpressions.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-09-03 13:56:03 +02:00
Yuanjian Li 95f1e9549b [SPARK-32782][SS] Refactor StreamingRelationV2 and move it to catalyst
### What changes were proposed in this pull request?
Move StreamingRelationV2 to the catalyst module and bind with the Table interface.

### Why are the changes needed?
Currently, the StreamingRelationV2 is bind with TableProvider. Since the V2 relation is not bound with `DataSource`, to make it more flexible and have better expansibility, it should be moved to the catalyst module and bound with the Table interface. We did a similar thing for DataSourceV2Relation.

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

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

Closes #29633 from xuanyuanking/SPARK-32782.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-03 16:04:36 +09:00
Kent Yao 1fba286407 [SPARK-32781][SQL] Non-ASCII characters are mistakenly omitted in the middle of intervals
### What changes were proposed in this pull request?

This PR fails the interval values parsing when they contain non-ASCII characters which are silently omitted right now.

e.g. the case below should be invalid

```
select interval 'interval中文 1 day'
```

### Why are the changes needed?

bugfix, intervals should fail when containing invalid characters

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

yes,

#### before

select interval 'interval中文 1 day'  results 1 day, now it fails with

```
org.apache.spark.sql.catalyst.parser.ParseException

Cannot parse the INTERVAL value: interval中文 1 day
```

### How was this patch tested?

new tests

Closes #29632 from yaooqinn/SPARK-32781.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-03 04:56:40 +00:00
Kousuke Saruta ad6b887541 [SPARK-32772][SQL] Reduce log messages for spark-sql CLI
### What changes were proposed in this pull request?

This PR reduces log messages for spark-sql CLI like spark-shell and pyspark CLI.

### Why are the changes needed?

When we launch spark-sql CLI, too many log messages are shown and it's sometimes difficult to find the result of query.
```
spark-sql> SELECT now();
20/09/02 00:11:45 INFO CodeGenerator: Code generated in 10.121625 ms
20/09/02 00:11:45 INFO SparkContext: Starting job: main at NativeMethodAccessorImpl.java:0
20/09/02 00:11:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 1 output partitions
20/09/02 00:11:45 INFO DAGScheduler: Final stage: ResultStage 0 (main at NativeMethodAccessorImpl.java:0)
20/09/02 00:11:45 INFO DAGScheduler: Parents of final stage: List()
20/09/02 00:11:45 INFO DAGScheduler: Missing parents: List()
20/09/02 00:11:45 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/09/02 00:11:45 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 6.3 KiB, free 366.3 MiB)
20/09/02 00:11:45 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 3.2 KiB, free 366.3 MiB)
20/09/02 00:11:45 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 192.168.1.204:42615 (size: 3.2 KiB, free: 366.3 MiB)
20/09/02 00:11:45 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1348
20/09/02 00:11:45 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0))
20/09/02 00:11:45 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0
20/09/02 00:11:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (192.168.1.204, executor driver, partition 0, PROCESS_LOCAL, 7561 bytes) taskResourceAssignments Map()
20/09/02 00:11:45 INFO Executor: Running task 0.0 in stage 0.0 (TID 0)
20/09/02 00:11:45 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1446 bytes result sent to driver
20/09/02 00:11:45 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 238 ms on 192.168.1.204 (executor driver) (1/1)
20/09/02 00:11:45 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool
20/09/02 00:11:45 INFO DAGScheduler: ResultStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 0.343 s
20/09/02 00:11:45 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job
20/09/02 00:11:45 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished
20/09/02 00:11:45 INFO DAGScheduler: Job 0 finished: main at NativeMethodAccessorImpl.java:0, took 0.377489 s
2020-09-02 00:11:45.07
Time taken: 0.704 seconds, Fetched 1 row(s)
20/09/02 00:11:45 INFO SparkSQLCLIDriver: Time taken: 0.704 seconds, Fetched 1 row(s)
```

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

Yes. Log messages are reduced for spark-sql CLI like as follows.
```
20/09/02 00:34:51 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/09/02 00:34:53 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/09/02 00:34:53 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
20/09/02 00:34:55 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
20/09/02 00:34:55 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore kou192.168.1.204
Spark master: local[*], Application Id: local-1598974492822
spark-sql> SELECT now();
2020-09-02 00:35:05.258
Time taken: 2.299 seconds, Fetched 1 row(s)
```

### How was this patch tested?

Launched spark-sql CLI and confirmed that log messages are reduced as I paste above.

Closes #29619 from sarutak/suppress-log-for-spark-sql.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-02 13:31:06 -07:00
angerszhu 5e6173ebef [SPARK-31670][SQL] Trim unnecessary Struct field alias in Aggregate/GroupingSets
### What changes were proposed in this pull request?
Struct field both in GROUP BY and Aggregate Expresison with CUBE/ROLLUP/GROUPING SET will failed when analysis.

```
test("SPARK-31670") {
  withTable("t1") {
      sql(
        """
          |CREATE TEMPORARY VIEW t(a, b, c) AS
          |SELECT * FROM VALUES
          |('A', 1, NAMED_STRUCT('row_id', 1, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 1}')),
          |('A', 2, NAMED_STRUCT('row_id', 2, 'json_string', '{"i": 2}')),
          |('B', 1, NAMED_STRUCT('row_id', 3, 'json_string', '{"i": 1}')),
          |('C', 3, NAMED_STRUCT('row_id', 4, 'json_string', '{"i": 1}'))
        """.stripMargin)

      checkAnswer(
        sql(
          """
            |SELECT a, c.json_string, SUM(b)
            |FROM t
            |GROUP BY a, c.json_string
            |WITH CUBE
            |""".stripMargin),
        Row("A", "{\"i\": 1}", 3) :: Row("A", "{\"i\": 2}", 2) :: Row("A", null, 5) ::
          Row("B", "{\"i\": 1}", 1) :: Row("B", null, 1) ::
          Row("C", "{\"i\": 1}", 3) :: Row("C", null, 3) ::
          Row(null, "{\"i\": 1}", 7) :: Row(null, "{\"i\": 2}", 2) :: Row(null, null, 9) :: Nil)

  }
}
```
Error 
```
[info] - SPARK-31670 *** FAILED *** (2 seconds, 857 milliseconds)
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: expression 't.`c`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
[info]   Aggregate [a#247, json_string#248, spark_grouping_id#246L], [a#247, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
[info]   +- Expand [List(a#221, b#222, c#223, a#244, json_string#245, 0), List(a#221, b#222, c#223, a#244, null, 1), List(a#221, b#222, c#223, null, json_string#245, 2), List(a#221, b#222, c#223, null, null, 3)], [a#221, b#222, c#223, a#247, json_string#248, spark_grouping_id#246L]
[info]      +- Project [a#221, b#222, c#223, a#221 AS a#244, c#223.json_string AS json_string#245]
[info]         +- SubqueryAlias t
[info]            +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
[info]               +- Project [col1#218, col2#219, col3#220]
[info]                  +- LocalRelation [col1#218, col2#219, col3#220]
[info]
```
For Struct type Field, when we resolve it, it will construct with Alias. When struct field in GROUP BY with CUBE/ROLLUP etc,  struct field in groupByExpression and aggregateExpression will be resolved with different exprId as below
```
'Aggregate [cube(a#221, c#223.json_string AS json_string#240)], [a#221, c#223.json_string AS json_string#241, sum(cast(b#222 as bigint)) AS sum(b)#243L]
+- SubqueryAlias t
   +- Project [col1#218 AS a#221, col2#219 AS b#222, col3#220 AS c#223]
      +- Project [col1#218, col2#219, col3#220]
         +- LocalRelation [col1#218, col2#219, col3#220]
```
This makes `ResolveGroupingAnalytics.constructAggregateExprs()` failed to replace aggreagteExpression use expand groupByExpression attribute since there exprId is not same. then error happened.

### Why are the changes needed?
Fix analyze bug

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

### How was this patch tested?
Added UT

Closes #28490 from AngersZhuuuu/SPARK-31670.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-02 13:49:09 +00:00
Zhenhua Wang 03afbc8820 [SPARK-32739][SQL] Support prune right for left semi join in DPP
### What changes were proposed in this pull request?

Currently in DPP, left semi can only prune left, this pr makes it also support prune right.

### Why are the changes needed?

A minor improvement for DPP.

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

No.

### How was this patch tested?

Add a test case.

Closes #29582 from wzhfy/dpp_support_leftsemi_pruneRight.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-02 21:34:49 +08:00
Karol Chmist 7511e43c50 [SPARK-32756][SQL] Fix CaseInsensitiveMap usage for Scala 2.13
### What changes were proposed in this pull request?

This is a follow-up of #29160. This allows Spark SQL project to compile for Scala 2.13.

### Why are the changes needed?

It's needed for #28545

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

No

### How was this patch tested?

I compiled with Scala 2.13. It fails in `Spark REPL` project, which will be fixed by #28545

Closes #29584 from karolchmist/SPARK-32364-scala-2.13.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-09-02 08:27:00 -05:00
Ali Smesseim 3cde392b69 [SPARK-31831][SQL][FOLLOWUP] Make the GetCatalogsOperationMock for HiveSessionImplSuite compile with the proper Hive version
### What changes were proposed in this pull request?
#29129 duplicated GetCatalogsOperationMock in the hive-version-specific subdirectories, otherwise profile hive-1.2 would not compile. We can prevent duplication of this class by shimming the required hive-version-specific types.

### Why are the changes needed?
This is a cleanup to avoid duplication of a mock class.

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

### How was this patch tested?
This patch only changes tests.

Closes #29549 from alismess-db/get-catalogs-operation-mock-use-shim.

Authored-by: Ali Smesseim <ali.smesseim@databricks.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2020-09-02 20:23:57 +08:00
angerszhu 55ce49ed28 [SPARK-32400][SQL][TEST][FOLLOWUP][TEST-MAVEN] Fix resource loading error in HiveScripTransformationSuite
### What changes were proposed in this pull request?
#29401 move `test_script.py` from sql/hive module to sql/core module, cause HiveScripTransformationSuite load resource issue.

### Why are the changes needed?
This issue cause jenkins test failed in mvn

spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3-jdk-11/
spark-master-test-maven-hadoop-3.2-hive-2.3:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-hive-2.3/
![image](https://user-images.githubusercontent.com/46485123/91681585-71285a80-eb81-11ea-8519-99fc9783d6b9.png)

![image](https://user-images.githubusercontent.com/46485123/91681010-aaf86180-eb7f-11ea-8dbb-61365a3b0ab4.png)

Error as below:
```
 Exception thrown while executing Spark plan:
 HiveScriptTransformation [a#349299, b#349300, c#349301, d#349302, e#349303], python /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py, [a#349309, b#349310, c#349311, d#349312, e#349313], ScriptTransformationIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim, )),List((field.delim, )),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
+- Project [_1#349288 AS a#349299, _2#349289 AS b#349300, _3#349290 AS c#349301, _4#349291 AS d#349302, _5#349292 AS e#349303]
   +- LocalTableScan [_1#349288, _2#349289, _3#349290, _4#349291, _5#349292]

 == Exception ==
 org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 18021.0 failed 1 times, most recent failure: Lost task 0.0 in stage 18021.0 (TID 37324) (192.168.10.31 executor driver): org.apache.spark.SparkException: Subprocess exited with status 2. Error: python: can't open file '/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/hive/file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/sql/core/target/spark-sql_2.12-3.1.0-SNAPSHOT-tests.jar!/test_script.py': [Errno 2] No such file or directory

 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate(BaseScriptTransformationExec.scala:180)
 at org.apache.spark.sql.execution.BaseScriptTransformationExec.checkFailureAndPropagate$(BaseScriptTransformationExec.scala:157)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec.checkFailureAndPropagate(HiveScriptTransformationExec.scala:49)
 at org.apache.spark.sql.hive.execution.HiveScriptTransformationExec$$anon$1.hasNext(HiveScriptTransformationExec.scala:110)
 at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
 at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
 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:127)
 at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:480)
 at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1426)
 at o
```
### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
Existed UT

Closes #29588 from AngersZhuuuu/SPARK-32400-FOLLOWUP.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:27:29 +09:00
liwensun f0851e95c6 [SPARK-32776][SS] Limit in streaming should not be optimized away by PropagateEmptyRelation
### What changes were proposed in this pull request?

PropagateEmptyRelation will not be applied to LIMIT operators in streaming queries.

### Why are the changes needed?

Right now, the limit operator in a streaming query may get optimized away when the relation is empty. This can be problematic for stateful streaming, as this empty batch will not write any state store files, and the next batch will fail when trying to read these state store files and throw a file not found error.

We should not let PropagateEmptyRelation optimize away the Limit operator for streaming queries.

This PR is intended as a small and safe fix for PropagateEmptyRelation. A fundamental fix that can prevent this from happening again in the future and in other optimizer rules is more desirable, but that's a much larger task.

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

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

Closes #29623 from liwensun/spark-32776.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 18:05:06 +09:00
Yuming Wang 54348dbd21 [SPARK-32767][SQL] Bucket join should work if spark.sql.shuffle.partitions larger than bucket number
### What changes were proposed in this pull request?

Bucket join should work if `spark.sql.shuffle.partitions` larger than bucket number, such as:
```scala
spark.range(1000).write.bucketBy(432, "id").saveAsTable("t1")
spark.range(1000).write.bucketBy(34, "id").saveAsTable("t2")
sql("set spark.sql.shuffle.partitions=600")
sql("set spark.sql.autoBroadcastJoinThreshold=-1")
sql("select * from t1 join t2 on t1.id = t2.id").explain()
```

Before this pr:
```
== Physical Plan ==
*(5) SortMergeJoin [id#26L], [id#27L], Inner
:- *(2) Sort [id#26L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#26L, 600), true
:     +- *(1) Filter isnotnull(id#26L)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.t1[id#26L] Batched: true, DataFilters: [isnotnull(id#26L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 432 out of 432
+- *(4) Sort [id#27L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#27L, 600), true
      +- *(3) Filter isnotnull(id#27L)
         +- *(3) ColumnarToRow
            +- FileScan parquet default.t2[id#27L] Batched: true, DataFilters: [isnotnull(id#27L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 34 out of 34
```

After this pr:
```
== Physical Plan ==
*(4) SortMergeJoin [id#26L], [id#27L], Inner
:- *(1) Sort [id#26L ASC NULLS FIRST], false, 0
:  +- *(1) Filter isnotnull(id#26L)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.t1[id#26L] Batched: true, DataFilters: [isnotnull(id#26L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 432 out of 432
+- *(3) Sort [id#27L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#27L, 432), true
      +- *(2) Filter isnotnull(id#27L)
         +- *(2) ColumnarToRow
            +- FileScan parquet default.t2[id#27L] Batched: true, DataFilters: [isnotnull(id#27L)], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct<id:bigint>, SelectedBucketsCount: 34 out of 34
```

### Why are the changes needed?

Spark 2.4 support this.

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

No.

### How was this patch tested?

Unit test.

Closes #29612 from wangyum/SPARK-32767.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-02 04:16:20 +00:00
Kousuke Saruta 812d0918a8 [SPARK-32771][DOCS] The example of expressions.Aggregator in Javadoc / Scaladoc is wrong
### What changes were proposed in this pull request?

This PR modifies an example for `expressions.Aggregator` in Javadoc and Scaladoc.
The definition of `bufferEncoder` and `outputEncoder` are added.

### Why are the changes needed?

To correct the example.
The current example is wrong and doesn't work because `bufferEncoder` and `outputEncoder` are not defined.

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

Yes.
Before this change, the scaladoc and javadoc are like as follows.
![wrong-example-java](https://user-images.githubusercontent.com/4736016/91897528-5ebf3580-ecd5-11ea-8d7b-e846b776ebbb.png)
![wrong-example](https://user-images.githubusercontent.com/4736016/91897509-58c95480-ecd5-11ea-81a3-98774083b689.png)

After this change, the docs are like as follows.
![fixed-example-java](https://user-images.githubusercontent.com/4736016/91897592-78607d00-ecd5-11ea-9e55-03fd9c9c6b54.png)
![fixed-example](https://user-images.githubusercontent.com/4736016/91897609-7c8c9a80-ecd5-11ea-837e-9dbcada6cd53.png)

### How was this patch tested?

Build with `build/sbt unidoc` and confirmed the generated javadoc/scaladoc and got the screenshots above.

Closes #29617 from sarutak/fix-aggregator-doc.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-02 10:03:07 +09:00
Zhenhua Wang 2a88a20271 [SPARK-32754][SQL][TEST] Unify to assertEqualJoinPlans for join reorder suites
### What changes were proposed in this pull request?

Now three join reorder suites(`JoinReorderSuite`, `StarJoinReorderSuite`, `StarJoinCostBasedReorderSuite`) all contain an `assertEqualPlans` method and the logic is almost the same. We can extract the method to a single place for code simplicity.

### Why are the changes needed?

To reduce code redundancy.

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

No.

### How was this patch tested?

Covered by existing tests.

Closes #29594 from wzhfy/unify_assertEqualPlans_joinReorder.

Authored-by: Zhenhua Wang <wzh_zju@163.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-09-01 09:08:35 -07:00
Linhong Liu a410658c9b [SPARK-32761][SQL] Allow aggregating multiple foldable distinct expressions
### What changes were proposed in this pull request?
For queries with multiple foldable distinct columns, since they will be eliminated during
execution, it's not mandatory to let `RewriteDistinctAggregates` handle this case. And
in the current code, `RewriteDistinctAggregates` *dose* miss some "aggregating with
multiple foldable distinct expressions" cases.
For example: `select count(distinct 2), count(distinct 2, 3)` will be missed.

But in the planner, this will trigger an error that "multiple distinct expressions" are not allowed.
As the foldable distinct columns can be eliminated finally, we can allow this in the aggregation
planner check.

### Why are the changes needed?
bug fix

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

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

Closes #29607 from linhongliu-db/SPARK-32761.

Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 13:04:24 +00:00
Wenchen Fan fea9360ae7 [SPARK-32757][SQL][FOLLOW-UP] Use child's output for canonicalization in SubqueryBroadcastExec
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/29601 , to fix a small mistake in `SubqueryBroadcastExec`. `SubqueryBroadcastExec.doCanonicalize` should canonicalize the build keys with the query output, not the `SubqueryBroadcastExec.output`.

### Why are the changes needed?

fix mistake

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

no

### How was this patch tested?

existing test

Closes #29610 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 12:54:40 +00:00
Huaxin Gao e1dbc85c72 [SPARK-32579][SQL] Implement JDBCScan/ScanBuilder/WriteBuilder
### What changes were proposed in this pull request?
Add JDBCScan, JDBCScanBuilder, JDBCWriteBuilder in Datasource V2 JDBC

### Why are the changes needed?
Complete Datasource V2 JDBC implementation

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

### How was this patch tested?
new tests

Closes #29396 from huaxingao/v2jdbc.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 07:23:20 +00:00
Wenchen Fan d2a5dad97c [SPARK-32757][SQL] Physical InSubqueryExec should be consistent with logical InSubquery
### What changes were proposed in this pull request?

`InSubquery` can be either single-column mode, or multi-column mode, depending on the output length of the subquery. For multi-column mode, the length of input `values` must match the subquery output length.

However, `InSubqueryExec` doesn't follow it and always be executed under single column mode. It's OK as it's only used by DPP, which looks up one key in one `InSubqueryExec`, so the multi-column mode is not needed. But it's better to make the physical and logical node consistent.

This PR updates `InSubqueryExec` to support multi-column mode, and also fix `SubqueryBroadcastExec` to report output correctly.

### Why are the changes needed?

Fix a potential bug.

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

no

### How was this patch tested?

existing tests

Closes #29601 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 07:19:43 +00:00
Kris Mok 6e5bc39e17 [SPARK-32624][SQL][FOLLOWUP] Fix regression in CodegenContext.addReferenceObj on nested Scala types
### What changes were proposed in this pull request?

Use `CodeGenerator.typeName()` instead of `Class.getCanonicalName()` in `CodegenContext.addReferenceObj()` for getting the runtime class name for an object.

### Why are the changes needed?

https://github.com/apache/spark/pull/29439 fixed a bug in `CodegenContext.addReferenceObj()` for `Array[Byte]` (i.e. Spark SQL's `BinaryType`) objects, but unfortunately it introduced a regression for some nested Scala types.

For example, for `implicitly[Ordering[UTF8String]]`, after that PR `CodegenContext.addReferenceObj()` would return `((null) references[0] /* ... */)`. The actual type for `implicitly[Ordering[UTF8String]]` is `scala.math.LowPriorityOrderingImplicits$$anon$3` in Scala 2.12.10, and `Class.getCanonicalName()` returns `null` for that class.

On the other hand, `Class.getName()` is safe to use for all non-array types, and Janino will happily accept the type name returned from `Class.getName()` for nested types. `CodeGenerator.typeName()` happens to do the right thing by correctly handling arrays and otherwise use `Class.getName()`. So it's a better alternative than `Class.getCanonicalName()`.

Side note: rule of thumb for using Java reflection in Spark: it may be tempting to use `Class.getCanonicalName()`, but for functions that may need to handle Scala types, please avoid it due to potential issues with nested Scala types.
Instead, use `Class.getName()` or utility functions in `org.apache.spark.util.Utils` (e.g. `Utils.getSimpleName()` or `Utils.getFormattedClassName()` etc).

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

No.

### How was this patch tested?

Added new unit test case for the regression case in `CodeGenerationSuite`.

Closes #29602 from rednaxelafx/spark-32624-followup.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-01 15:15:11 +09:00
Chao Sun 94d313b061 [SPARK-32721][SQL][FOLLOWUP] Simplify if clauses with null and boolean
### What changes were proposed in this pull request?

This is a follow-up on SPARK-32721 and PR #29567. In the previous PR we missed two more cases that can be optimized:
```
if(p, false, null) ==> and(not(p), null)
if(p, true, null) ==> or(p, null)
```

### Why are the changes needed?

By transforming if to boolean conjunctions or disjunctions, we can enable more filter pushdown to datasources.

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

No

### How was this patch tested?

Added unit tests.

Closes #29603 from sunchao/SPARK-32721-2.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-09-01 06:06:25 +00:00
Yuming Wang a701bc79e3 [SPARK-32659][SQL][FOLLOWUP] Improve test for pruning DPP on non-atomic type
### What changes were proposed in this pull request?

Improve test for pruning DPP on non-atomic type:
- Avoid creating new partition tables. This may take 30 seconds..
- Add test `array` type.

### Why are the changes needed?

Improve test.

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

No

### How was this patch tested?

N/A

Closes #29595 from wangyum/SPARK-32659-test.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-09-01 05:51:04 +00:00
Chao Sun 1453a09a63 [SPARK-32721][SQL] Simplify if clauses with null and boolean
### What changes were proposed in this pull request?

The following if clause:
```sql
if(p, null, false)
```
can be simplified to:
```sql
and(p, null)
```
Similarly, the clause:
```sql
if(p, null, true)
```
can be simplified to
```sql
or(not(p), null)
```
iff the predicate `p` is non-nullable, i.e., can be evaluated to either true or false, but not null.

### Why are the changes needed?

Converting if to or/and clauses can better push filters down.

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

No.

### How was this patch tested?

Unit tests.

Closes #29567 from sunchao/SPARK-32721.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-08-31 20:59:54 +00:00
Huaxin Gao 806140de40 [SPARK-32592][SQL] Make DataFrameReader.table take the specified options
### What changes were proposed in this pull request?
pass specified options in DataFrameReader.table to JDBCTableCatalog.loadTable

### Why are the changes needed?
Currently, `DataFrameReader.table` ignores the specified options. The options specified like the following are lost.
```
    val df = spark.read
      .option("partitionColumn", "id")
      .option("lowerBound", "0")
      .option("upperBound", "3")
      .option("numPartitions", "2")
      .table("h2.test.people")
```
We need to make `DataFrameReader.table` take the specified options.

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

### How was this patch tested?
Manually test for now. Will add a test after V2 JDBC read is implemented.

Closes #29535 from huaxingao/table_options.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-31 13:21:15 +00:00
Cheng Su ce473b223a [SPARK-32740][SQL] Refactor common partitioning/distribution logic to BaseAggregateExec
### What changes were proposed in this pull request?

For all three different aggregate physical operator: `HashAggregateExec`, `ObjectHashAggregateExec` and `SortAggregateExec`, they have same `outputPartitioning` and `requiredChildDistribution` logic. Refactor these same logic into their super class `BaseAggregateExec` to avoid code duplication and future bugs (similar to `HashJoin` and `ShuffledJoin`).

### Why are the changes needed?

Reduce duplicated code across classes and prevent future bugs if we only update one class but forget another. We already did similar refactoring for join (`HashJoin` and `ShuffledJoin`).

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

No.

### How was this patch tested?

Existing unit tests as this is pure refactoring and no new logic added.

Closes #29583 from c21/aggregate-refactor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-31 15:43:13 +09:00
Kent Yao 6dacba7fa0 [SPARK-32733][SQL] Add extended information - arguments/examples/since/notes of expressions to the remarks field of GetFunctionsOperation
### What changes were proposed in this pull request?

This PR adds extended information of a function including arguments, examples, notes and the since field to the SparkGetFunctionOperation

### Why are the changes needed?

better user experience, it will help JDBC users to have a better understanding of our builtin functions

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

Yes, BI tools and JDBC users will get full information on a spark function instead of only fragmentary usage info.

e.g. date_part

#### before

```
date_part(field, source) - Extracts a part of the date/timestamp or interval source.
```
#### after

```
    Usage:
      date_part(field, source) - Extracts a part of the date/timestamp or interval source.

    Arguments:
      * field - selects which part of the source should be extracted, and supported string values are as same as the fields of the equivalent function `EXTRACT`.
      * source - a date/timestamp or interval column from where `field` should be extracted

    Examples:
      > SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456');
       2019
      > SELECT date_part('week', timestamp'2019-08-12 01:00:00.123456');
       33
      > SELECT date_part('doy', DATE'2019-08-12');
       224
      > SELECT date_part('SECONDS', timestamp'2019-10-01 00:00:01.000001');
       1.000001
      > SELECT date_part('days', interval 1 year 10 months 5 days);
       5
      > SELECT date_part('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds);
       30.001001

    Note:
      The date_part function is equivalent to the SQL-standard function `EXTRACT(field FROM source)`

    Since: 3.0.0

```

### How was this patch tested?

New tests

Closes #29577 from yaooqinn/SPARK-32733.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-31 11:03:01 +09:00
Udbhav30 065f17386d [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29552 from Udbhav30/truncate.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-30 10:25:32 -07:00
Cheng Su cfe012a431 [SPARK-32629][SQL] Track metrics of BitSet/OpenHashSet in full outer SHJ
### What changes were proposed in this pull request?

This is followup from https://github.com/apache/spark/pull/29342, where to do two things:
* Per https://github.com/apache/spark/pull/29342#discussion_r470153323, change from java `HashSet` to spark in-house `OpenHashSet` to track matched rows for non-unique join keys. I checked `OpenHashSet` implementation which is built from a key index (`OpenHashSet._bitset` as `BitSet`) and key array (`OpenHashSet._data` as `Array`). Java `HashSet` is built from `HashMap`, which stores value in `Node` linked list and by theory should have taken more memory than `OpenHashSet`. Reran the same benchmark query used in https://github.com/apache/spark/pull/29342, and verified the query has similar performance here between `HashSet` and `OpenHashSet`.
* Track metrics of the extra data structure `BitSet`/`OpenHashSet` for full outer SHJ. This depends on above thing, because there seems no easy way to get java `HashSet` memory size.

### Why are the changes needed?

To better surface the memory usage for full outer SHJ more accurately.
This can help users/developers to debug/improve full outer SHJ.

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

No.

### How was this patch tested?

Added unite test in `SQLMetricsSuite.scala` .

Closes #29566 from c21/add-metrics.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-30 07:01:33 +09:00
Wenchen Fan ccc0250a08 [SPARK-32718][SQL] Remove unnecessary keywords for interval units
### What changes were proposed in this pull request?

Remove the YEAR, MONTH, DAY, HOUR, MINUTE, SECOND keywords. They are not useful in the parser, as we need to support plural like YEARS, so the parser has to accept the general identifier as interval unit anyway.

### Why are the changes needed?

These keywords are reserved in ANSI. If Spark has these keywords, then they become reserved under ANSI mode. This makes Spark not able to run TPCDS queries as they use YEAR as alias name.

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

No

### How was this patch tested?

Added `TPCDSQueryANSISuite`, to make sure Spark with ANSI mode can run TPCDS queries.

Closes #29560 from cloud-fan/keyword.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-29 14:06:01 -07:00
Chen Zhang 58f87b3178 [SPARK-32639][SQL] Support GroupType parquet mapkey field
### What changes were proposed in this pull request?
Remove the assertion in ParquetSchemaConverter that the parquet mapKey field must be PrimitiveType.

### Why are the changes needed?
There is a parquet file in the attachment of [SPARK-32639](https://issues.apache.org/jira/browse/SPARK-32639), and the MessageType recorded in the file is:
```
message parquet_schema {
  optional group value (MAP) {
    repeated group key_value {
      required group key {
        optional binary first (UTF8);
        optional binary middle (UTF8);
        optional binary last (UTF8);
      }
      optional binary value (UTF8);
    }
  }
}
```

Use `spark.read.parquet("000.snappy.parquet")` to read the file. Spark will throw an exception when converting Parquet MessageType to Spark SQL StructType:

> AssertionError(Map key type is expected to be a primitive type, but found...)

Use `spark.read.schema("value MAP<STRUCT<first:STRING, middle:STRING, last:STRING>, STRING>").parquet("000.snappy.parquet")` to read the file, spark returns the correct result .

According to the parquet project document (https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps), the mapKey in the parquet format does not need to be a primitive type.

Note: This parquet file is not written by spark, because spark will write additional sparkSchema string information in the parquet file. When Spark reads, it will directly use the additional sparkSchema information in the file instead of converting Parquet MessageType to Spark SQL StructType.

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

### How was this patch tested?
Added a unit test case

Closes #29451 from izchen/SPARK-32639.

Authored-by: Chen Zhang <izchen@126.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-28 16:51:00 +00:00
Takeshi Yamamuro 0cb91b8c18 [SPARK-32704][SQL] Logging plan changes for execution
### What changes were proposed in this pull request?

Since we only log plan changes for analyzer/optimizer now, this PR intends to add code to log plan changes in the preparation phase in `QueryExecution` for execution.
```
scala> spark.sql("SET spark.sql.optimizer.planChangeLog.level=WARN")
scala> spark.range(10).groupBy("id").count().queryExecution.executedPlan
...
20/08/26 09:32:36 WARN PlanChangeLogger:
=== Applying Rule org.apache.spark.sql.execution.CollapseCodegenStages ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)

20/08/26 09:32:36 WARN PlanChangeLogger:
=== Result of Batch Preparations ===
!HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])              *(1) HashAggregate(keys=[id#19L], functions=[count(1)], output=[id#19L, count#23L])
!+- HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])   +- *(1) HashAggregate(keys=[id#19L], functions=[partial_count(1)], output=[id#19L, count#27L])
!   +- Range (0, 10, step=1, splits=4)                                                          +- *(1) Range (0, 10, step=1, splits=4)
```

### Why are the changes needed?

Easy debugging for executed plans

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

No.

### How was this patch tested?

Added unit tests.

Closes #29544 from maropu/PlanLoggingInPreparations.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-28 16:35:47 +00:00
Kent Yao 0626901bcb [SPARK-32729][SQL][DOCS] Add missing since version for math functions
### What changes were proposed in this pull request?

Add missing since version for math functions, including
SPARK-8223 shiftright/shiftleft
SPARK-8215 pi
SPARK-8212 e
SPARK-6829 sin/asin/sinh/cos/acos/cosh/tan/atan/tanh/ceil/floor/rint/cbrt/signum/isignum/Fsignum/Lsignum/degrees/radians/log/log10/log1p/exp/expm1/pow/hypot/atan2
SPARK-8209 conv
SPARK-8213 factorial
SPARK-20751 cot
SPARK-2813 sqrt
SPARK-8227 unhex
SPARK-8218 log(a,b)
SPARK-8207 bin
SPARK-8214 hex
SPARK-8206 round
SPARK-14614 bround

### Why are the changes needed?

fix SQL docs
### Does this PR introduce _any_ user-facing change?

yes, doc updated

### How was this patch tested?

passing doc generation.

Closes #29571 from yaooqinn/minor.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-29 00:30:31 +09:00
yi.wu c3b9404253 [SPARK-32717][SQL] Add a AQEOptimizer for AdaptiveSparkPlanExec
### What changes were proposed in this pull request?

This PR proposes to add a specific `AQEOptimizer` for the `AdaptiveSparkPlanExec` instead of implementing an anonymous `RuleExecutor`. At the same time, this PR also adds the configuration `spark.sql.adaptive.optimizer.excludedRules`, which follows the same pattern of `Optimizer`, to make the `AQEOptimizer` more flexible for users and developers.

### Why are the changes needed?

Currently, `AdaptiveSparkPlanExec` has implemented an anonymous `RuleExecutor` to apply the AQE optimize rules on the plan. However, the anonymous class usually could be inconvenient to maintain and extend for the long term.

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

No.

### How was this patch tested?

It's a pure refactor so pass existing tests should be ok.

Closes #29559 from Ngone51/impro-aqe-optimizer.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 21:23:53 +09:00
Jungtaek Lim (HeartSaVioR) 73bfed3633 [SPARK-28612][SQL][FOLLOWUP] Correct method doc of DataFrameWriterV2.replace()
### What changes were proposed in this pull request?

This patch corrects the method doc of DataFrameWriterV2.replace() which explanation of exception is described oppositely.

### Why are the changes needed?

The method doc is incorrect.

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

No.

### How was this patch tested?

Only doc change.

Closes #29568 from HeartSaVioR/SPARK-28612-FOLLOWUP-fix-doc-nit.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 15:14:57 +09:00
HyukjinKwon c154629171 [SPARK-32183][DOCS][PYTHON] User Guide - PySpark Usage Guide for Pandas with Apache Arrow
### What changes were proposed in this pull request?

This PR proposes to move Arrow usage guide from Spark documentation site to PySpark documentation site (at "User Guide").

Here is the demo for reviewing quicker: https://hyukjin-spark.readthedocs.io/en/stable/user_guide/arrow_pandas.html

### Why are the changes needed?

To have a single place for PySpark users, and better documentation.

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

Yes, it will move https://spark.apache.org/docs/latest/sql-pyspark-pandas-with-arrow.html to our PySpark documentation.

### How was this patch tested?

```bash
cd docs
SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll serve --watch
```

and

```bash
cd python/docs
make clean html
```

Closes #29548 from HyukjinKwon/SPARK-32183.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-28 15:09:06 +09:00
Liang-Chi Hsieh d6c095c92c [SPARK-32693][SQL] Compare two dataframes with same schema except nullable property
### What changes were proposed in this pull request?

This PR changes key data types check in `HashJoin` to use `sameType`.

### Why are the changes needed?

Looks at the resolving condition of `SetOperation`, it requires only each left data types should be `sameType` as the right ones. Logically the `EqualTo` expression in equi-join, also requires only left data type `sameType` as right data type. Then `HashJoin` requires left keys data type exactly the same as right keys data type, looks not reasonable.

It makes inconsistent results when doing `except` between two dataframes.

If two dataframes don't have nested fields, even their field nullable property different, `HashJoin` passes the key type check because it checks field individually so field nullable property is ignored.

If two dataframes have nested fields like struct, `HashJoin` fails the key type check because now it compare two struct types and nullable property now affects.

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

Yes. Making consistent `except` operation between dataframes.

### How was this patch tested?

Unit test.

Closes #29555 from viirya/SPARK-32693.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-28 10:32:23 +09:00
Kent Yao f14f3742e0 [SPARK-32696][SQL][TEST-HIVE1.2][TEST-HADOOP2.7] Get columns operation should handle interval column properly
### What changes were proposed in this pull request?

This PR let JDBC clients identify spark interval columns properly.

### Why are the changes needed?

JDBC users can query interval values through thrift server, create views with interval columns, e.g.
```sql
CREATE global temp view view1 as select interval 1 day as i;
```
but when they want to get the details of the columns of view1, the will fail with `Unrecognized type name: INTERVAL`

```
Caused by: java.lang.IllegalArgumentException: Unrecognized type name: INTERVAL
	at org.apache.hadoop.hive.serde2.thrift.Type.getType(Type.java:170)
	at org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils$.toJavaSQLType(ThriftserverShimUtils.scala:53)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$addToRowSet$1(SparkGetColumnsOperation.scala:157)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:102)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.addToRowSet(SparkGetColumnsOperation.scala:149)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$6(SparkGetColumnsOperation.scala:113)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$6$adapted(SparkGetColumnsOperation.scala:112)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$5(SparkGetColumnsOperation.scala:112)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.$anonfun$runInternal$5$adapted(SparkGetColumnsOperation.scala:111)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.sql.hive.thriftserver.SparkGetColumnsOperation.runInternal(SparkGetColumnsOperation.scala:111)
	... 34 more
```
### Does this PR introduce _any_ user-facing change?

YES,
#### before
![image](https://user-images.githubusercontent.com/8326978/91162239-6cd1ec80-e6fe-11ea-8c2c-914ddb325c4e.png)

#### after
![image](https://user-images.githubusercontent.com/8326978/91162025-1a90cb80-e6fe-11ea-94c4-03a6f2ec296b.png)

### How was this patch tested?

new tests

Closes #29539 from yaooqinn/SPARK-32696.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:52:34 +00:00
xuewei.linxuewei eb379766f4 [SPARK-32705][SQL] Fix serialization issue for EmptyHashedRelation
### What changes were proposed in this pull request?
Currently, EmptyHashedRelation and HashedRelationWithAllNullKeys is an object, and it will cause JavaDeserialization Exception as following

```
20/08/26 11:13:30 WARN [task-result-getter-2] TaskSetManager: Lost task 34.0 in stage 57.0 (TID 18076, emr-worker-5.cluster-183257, executor 18): java.io.InvalidClassException: org.apache.spark.sql.execution.joins.EmptyHashedRelation$; no valid constructor
        at java.io.ObjectStreamClass$ExceptionInfo.newInvalidClassException(ObjectStreamClass.java:169)
        at java.io.ObjectStreamClass.checkDeserialize(ObjectStreamClass.java:874)
        at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2042)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1572)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:430)
        at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76)
        at org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:328)
```

This PR includes

* Using case object instead to fix serialization issue.
* Also change EmptyHashedRelation not to extend NullAwareHashedRelation since it's already being used in other non-NAAJ joins.

### Why are the changes needed?
It will cause BHJ failed when buildSide is Empty and BHJ(NAAJ) failed when buildSide with null partition keys.

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

### How was this patch tested?
* Existing UT.
* Run entire TPCDS for E2E coverage.

Closes #29547 from leanken/leanken-SPARK-32705.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:24:42 +00:00
Terry Kim baaa756dee [SPARK-32516][SQL][FOLLOWUP] 'path' option cannot coexist with path parameter for DataFrameWriter.save(), DataStreamReader.load() and DataStreamWriter.start()
### What changes were proposed in this pull request?

This is a follow up PR to #29328 to apply the same constraint where `path` option cannot coexist with path parameter to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

### Why are the changes needed?

The current behavior silently overwrites the `path` option if path parameter is passed to `DataFrameWriter.save()`, `DataStreamReader.load()` and `DataStreamWriter.start()`.

For example,
```
Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
```
will write the result to `/tmp/path2`.

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

Yes, if `path` option coexists with path parameter to any of the above methods, it will throw `AnalysisException`:
```
scala> Seq(1).toDF.write.option("path", "/tmp/path1").parquet("/tmp/path2")
org.apache.spark.sql.AnalysisException: There is a 'path' option set and save() is called with a  path parameter. Either remove the path option, or call save() without the parameter. To ignore this check, set 'spark.sql.legacy.pathOptionBehavior.enabled' to 'true'.;
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added new tests.

Closes #29543 from imback82/path_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-27 06:21:04 +00:00
Dongjoon Hyun 2dee4352a0 Revert "[SPARK-32481][CORE][SQL] Support truncate table to move data to trash"
This reverts commit 5c077f0580.
2020-08-26 11:24:35 -07:00
Yuming Wang a8b568800e [SPARK-32659][SQL] Fix the data issue when pruning DPP on non-atomic type
### What changes were proposed in this pull request?

Use `InSet` expression to fix data issue when pruning DPP on non-atomic type. for example:
   ```scala
    spark.range(1000)
    .select(col("id"), col("id").as("k"))
    .write
    .partitionBy("k")
    .format("parquet")
    .mode("overwrite")
    .saveAsTable("df1");

   spark.range(100)
   .select(col("id"), col("id").as("k"))
   .write
   .partitionBy("k")
   .format("parquet")
   .mode("overwrite")
   .saveAsTable("df2")

   spark.sql("set spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio=2")
   spark.sql("set spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly=false")
   spark.sql("SELECT df1.id, df2.k FROM df1 JOIN df2 ON struct(df1.k) = struct(df2.k) AND df2.id < 2").show
   ```
   It should return two records, but it returns empty.

### Why are the changes needed?

Fix data issue

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

No.

### How was this patch tested?

Add new unit test.

Closes #29475 from wangyum/SPARK-32659.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-26 06:57:43 +00:00
Udbhav30 5c077f0580 [SPARK-32481][CORE][SQL] Support truncate table to move data to trash
### What changes were proposed in this pull request?
Instead of deleting the data, we can move the data to trash.
Based on the configuration provided by the user it will be deleted permanently from the trash.

### Why are the changes needed?
Instead of directly deleting the data, we can provide flexibility to move data to the trash and then delete it permanently.

### Does this PR introduce _any_ user-facing change?
Yes, After truncate table the data is not permanently deleted now.
It is first moved to the trash and then after the given time deleted permanently;

### How was this patch tested?
new UTs added

Closes #29387 from Udbhav30/tuncateTrash.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-25 23:38:43 -07:00
yi.wu f510d21e93 [SPARK-32466][FOLLOW-UP][TEST][SQL] Regenerate the golden explain file for PlanStabilitySuite
### What changes were proposed in this pull request?

This PR regenerates the golden explain file based on the fix: https://github.com/apache/spark/pull/29537

### Why are the changes needed?

Eliminates the personal related information (e.g., local directories) in the explain plan.

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

No.

### How was this patch tested?

Checked manually.

Closes #29546 from Ngone51/follow-up-gen-golden-file.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-26 14:46:14 +09:00
Yuming Wang 1354cf0842 [SPARK-32620][SQL] Reset the numPartitions metric when DPP is enabled
### What changes were proposed in this pull request?

This pr reset the `numPartitions` metric when DPP is enabled. Otherwise, it is always a [static value](18cac6a9f0/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala (L215)).

### Why are the changes needed?

Fix metric issue.

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

No.

### How was this patch tested?

Unit test and manual test

For [this test case](18cac6a9f0/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala (L252-L280)).

Before this pr:
![image](https://user-images.githubusercontent.com/5399861/90301798-9310b480-ded4-11ea-9294-49bcaba46f83.png)

After this pr:
![image](https://user-images.githubusercontent.com/5399861/90301709-0fef5e80-ded4-11ea-942d-4d45d1dd15bc.png)

Closes #29436 from wangyum/SPARK-32620.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-08-25 18:46:10 -07:00
Sean Owen a9d4e60a90 [SPARK-32614][SQL] Don't apply comment processing if 'comment' unset for CSV
### What changes were proposed in this pull request?

Spark's CSV source can optionally ignore lines starting with a comment char. Some code paths check to see if it's set before applying comment logic (i.e. not set to default of `\0`), but many do not, including the one that passes the option to Univocity. This means that rows beginning with a null char were being treated as comments even when 'disabled'.

### Why are the changes needed?

To avoid dropping rows that start with a null char when this is not requested or intended. See JIRA for an example.

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

Nothing beyond the effect of the bug fix.

### How was this patch tested?

Existing tests plus new test case.

Closes #29516 from srowen/SPARK-32614.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-26 00:25:58 +09:00
yi.wu b78b776c9e [SPARK-32466][SQL][FOLLOW-UP] Normalize Location info in explain plan
### What changes were proposed in this pull request?

1. Extract `SQLQueryTestSuite.replaceNotIncludedMsg` to `PlanTest`.

2. Reuse `replaceNotIncludedMsg` to normalize the explain plan that generated in `PlanStabilitySuite`.

### Why are the changes needed?

This's a follow-up of https://github.com/apache/spark/pull/29270.
Eliminates the personal related information (e.g., local directories) in the explain plan.

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

No.

### How was this patch tested?

Updated test.

Closes #29537 from Ngone51/follow-up-plan-stablity.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 21:03:44 +09:00
Kent Yao c26a97637f Revert "[SPARK-32412][SQL] Unify error handling for spark thrift serv…
…er operations"

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

This reverts commit 510a1656e6.

### Why are the changes needed?

see https://github.com/apache/spark/pull/29204#discussion_r475716547

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

NO

### How was this patch tested?

pass ci tools

Closes #29531 from yaooqinn/revert.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-25 05:57:14 +00:00
fqaiser94@gmail.com 3f1e56d4ca [SPARK-32641][SQL] withField + getField should return null if original struct was null
### What changes were proposed in this pull request?

There is a bug in the way the optimizer rule in `SimplifyExtractValueOps` is currently written in master branch which yields incorrect results in scenarios like the following:
```
sql("SELECT CAST(NULL AS struct<a:int,b:int>) struct_col")
.select($"struct_col".withField("d", lit(4)).getField("d").as("d"))

// currently returns this:
+---+
|d  |
+---+
|4  |
+---+

// when in fact it should return this:
+----+
|d   |
+----+
|null|
+----+
```
The changes in this PR will fix this bug.

### Why are the changes needed?

To fix the aforementioned bug. Optimizer rules should improve the performance of the  query but yield exactly the same results.

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

Yes, this bug will no longer occur.
That said, this isn't something to be concerned about as this bug was introduced in Spark 3.1 and Spark 3.1 has yet to be released.

### How was this patch tested?

Unit tests were added. Jenkins must pass them.

Closes #29522 from fqaiser94/SPARK-32641.

Authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-25 04:59:37 +00:00
Liang-Chi Hsieh cee48a9661 [SPARK-32646][SQL][TEST-HADOOP2.7][TEST-HIVE1.2] ORC predicate pushdown should work with case-insensitive analysis
### What changes were proposed in this pull request?

This PR proposes to fix ORC predicate pushdown under case-insensitive analysis case. The field names in pushed down predicates don't need to match in exact letter case with physical field names in ORC files, if we enable case-insensitive analysis.

This is re-submitted for #29457.  Because #29457 has a hive-1.2 error and there were some tests failed with hive-1.2 profile at the same time, #29457 was reverted to unblock others.

### Why are the changes needed?

Currently ORC predicate pushdown doesn't work with case-insensitive analysis. A predicate "a < 0" cannot pushdown to ORC file with field name "A" under case-insensitive analysis.

But Parquet predicate pushdown works with this case. We should make ORC predicate pushdown work with case-insensitive analysis too.

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

Yes, after this PR, under case-insensitive analysis, ORC predicate pushdown will work.

### How was this patch tested?

Unit tests.

Closes #29530 from viirya/fix-orc-pushdown.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 13:47:52 +09:00
Nicholas Chammas f540031419 [SPARK-31000][PYTHON][SQL] Add ability to set table description via Catalog.createTable()
### What changes were proposed in this pull request?

This PR enhances `Catalog.createTable()` to allow users to set the table's description. This corresponds to the following SQL syntax:

```sql
CREATE TABLE ...
COMMENT 'this is a fancy table';
```

### Why are the changes needed?

This brings the Scala/Python catalog APIs a bit closer to what's already possible via SQL.

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

Yes, it adds a new parameter to `Catalog.createTable()`.

### How was this patch tested?

Existing unit tests:

```sh
./python/run-tests \
  --python-executables python3.7 \
  --testnames 'pyspark.sql.tests.test_catalog,pyspark.sql.tests.test_context'
```

```
$ ./build/sbt
testOnly org.apache.spark.sql.internal.CatalogSuite org.apache.spark.sql.CachedTableSuite org.apache.spark.sql.hive.MetastoreDataSourcesSuite org.apache.spark.sql.hive.execution.HiveDDLSuite
```

Closes #27908 from nchammas/SPARK-31000-table-description.

Authored-by: Nicholas Chammas <nicholas.chammas@liveramp.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 13:42:31 +09:00
Yesheng Ma 3eee915b47 [MINOR][SQL] Add missing documentation for LongType mapping
### What changes were proposed in this pull request?

Added Java docs for Long data types in the Row class.

### Why are the changes needed?

The Long datatype is somehow missing in Row.scala's `apply` and `get` methods.

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

No

### How was this patch tested?

Existing UTs.

Closes #29534 from yeshengm/docs-fix.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 11:20:01 +09:00
yangjie01 a30bb0cfda [SPARK-32550][SQL][FOLLOWUP] Eliminate negative impact on HyperLogLogSuite
### What changes were proposed in this pull request?
Change to use `dataTypes.foreach` instead of get the element use specified index in `def this(dataTypes: Seq[DataType]) `constructor of `SpecificInternalRow` because the random access performance is unsatisfactory if the input argument not a `IndexSeq`.

This pr followed srowen's  advice.

### Why are the changes needed?
I found that SPARK-32550 had some negative impact on performance, the typical cases is "deterministic cardinality estimation" in `HyperLogLogPlusPlusSuite` when rsd is 0.001, we found the code that is significantly slower is line 41 in `HyperLogLogPlusPlusSuite`: `new SpecificInternalRow(hll.aggBufferAttributes.map(_.dataType)) `

08b951b1cb/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala (L40-L44)

The size of "hll.aggBufferAttributes" in this case is 209716, the results of comparison before and after spark-32550 merged are as follows, The unit is ns:

  | After   SPARK-32550 createBuffer | After   SPARK-32550 end to end | Before   SPARK-32550 createBuffer | Before   SPARK-32550 end to end
-- | -- | -- | -- | --
rsd 0.001, n   1000 | 52715513243 | 53004810687 | 195807999 | 773977677
rsd 0.001, n   5000 | 51881246165 | 52519358215 | 13689949 | 249974855
rsd 0.001, n   10000 | 52234282788 | 52374639172 | 14199071 | 183452846
rsd 0.001, n   50000 | 55503517122 | 55664035449 | 15219394 | 584477125
rsd 0.001, n   100000 | 51862662845 | 52116774177 | 19662834 | 166483678
rsd 0.001, n   500000 | 51619226715 | 52183189526 | 178048012 | 16681330
rsd 0.001, n   1000000 | 54861366981 | 54976399142 | 226178708 | 18826340
rsd 0.001, n   5000000 | 52023602143 | 52354615149 | 388173579 | 15446409
rsd 0.001, n   10000000 | 53008591660 | 53601392304 | 533454460 | 16033032

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

### How was this patch tested?
`mvn test -pl sql/catalyst -DwildcardSuites=org.apache.spark.sql.catalyst.expressions.aggregate.HyperLogLogPlusPlusSuite -Dtest=none`

**Before**:

```
Run completed in 8 minutes, 18 seconds.
Total number of tests run: 5
Suites: completed 2, aborted 0
Tests: succeeded 5, failed 0, canceled 0, ignored 0, pending 0
```

**After**
```
Run completed in 7 seconds, 65 milliseconds.
Total number of tests run: 5
Suites: completed 2, aborted 0
Tests: succeeded 5, failed 0, canceled 0, ignored 0, pending 0
```

Closes #29529 from LuciferYang/revert-spark-32550.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-25 11:13:01 +09:00
Terry Kim e3a88a9767 [SPARK-32516][SQL] 'path' option cannot coexist with load()'s path parameters
### What changes were proposed in this pull request?

This PR proposes to make the behavior consistent for the `path` option when loading dataframes with a single path (e.g, `option("path", path).format("parquet").load(path)` vs. `option("path", path).parquet(path)`) by disallowing `path` option to coexist with `load`'s path parameters.

### Why are the changes needed?

The current behavior is inconsistent:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
+-----+
|value|
+-----+
|    1|
+-----+

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
+-----+
|value|
+-----+
|    1|
|    1|
+-----+
```

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

Yes, now if the `path` option is specified along with `load`'s path parameters, it would fail:
```scala
scala> Seq(1).toDF.write.mode("overwrite").parquet("/tmp/test")

scala> spark.read.option("path", "/tmp/test").format("parquet").load("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:232)
  ... 47 elided

scala> spark.read.option("path", "/tmp/test").parquet("/tmp/test").show
org.apache.spark.sql.AnalysisException: There is a path option set and load() is called with path parameters. Either remove the path option or move it into the load() parameters.;
  at org.apache.spark.sql.DataFrameReader.verifyPathOptionDoesNotExist(DataFrameReader.scala:310)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:250)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:778)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:756)
  ... 47 elided
```

The user can restore the previous behavior by setting `spark.sql.legacy.pathOptionBehavior.enabled` to `true`.

### How was this patch tested?

Added a test

Closes #29328 from imback82/dfw_option.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 16:30:30 +00:00
Cheng Su 08b951b1cb [SPARK-32649][SQL] Optimize BHJ/SHJ inner/semi join with empty hashed relation
### What changes were proposed in this pull request?

For broadcast hash join and shuffled hash join, whenever the build side hashed relation turns out to be empty. We don't need to execute stream side plan at all, and can return an empty iterator (for inner join and left semi join), because we know for sure that none of stream side rows can be outputted as there's no match.

### Why are the changes needed?

A very minor optimization for rare use case, but in case build side turns out to be empty, we can leverage it to short-cut stream side to save CPU and IO.

Example broadcast hash join query similar to `JoinBenchmark` with empty hashed relation:

```
  def broadcastHashJoinLongKey(): Unit = {
    val N = 20 << 20
    val M = 1 << 16

    val dim = broadcast(spark.range(0).selectExpr("id as k", "cast(id as string) as v"))
    codegenBenchmark("Join w long", N) {
      val df = spark.range(N).join(dim, (col("id") % M) === col("k"))
      assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined)
      df.noop()
    }
  }
```

Comparing wall clock time for enabling and disabling this PR (for non-codegen code path). Seeing like 8x improvement.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
Join w long:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
Join PR disabled                                    637            646          12         32.9          30.4       1.0X
Join PR enabled                                      77             78           2        271.8           3.7       8.3X
```

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

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29484 from c21/empty-relation.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-24 08:53:26 +00:00
Liang-Chi Hsieh 11c6a23c13 [SPARK-32352][SQL][FOLLOW-UP][TEST-HADOOP2.7][TEST-HIVE1.2] Exclude partition columns from data columns
### What changes were proposed in this pull request?

This PR fixes a bug of #29406. #29406 partially pushes down data filter even if it mixed in partition filters. But in some cases partition columns might be in data columns too. It will possibly push down a predicate with partition column to datasource.

### Why are the changes needed?

The test "org.apache.spark.sql.hive.orc.HiveOrcHadoopFsRelationSuite.save()/load() - partitioned table - simple queries - partition columns in data" is currently failed with hive-1.2 profile in master branch.

```
[info] - save()/load() - partitioned table - simple queries - partition columns in data *** FAILED *** (1 second, 457 milliseconds)
[info]   java.util.NoSuchElementException: key not found: p1
[info]   at scala.collection.immutable.Map$Map2.apply(Map.scala:138)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.buildLeafSearchArgument(OrcFilters.scala:250)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.convertibleFiltersHelper$1(OrcFilters.scala:143)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.$anonfun$convertibleFilters$4(OrcFilters.scala:146)
[info]   at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
[info]   at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
[info]   at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
[info]   at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:38)
[info]   at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
[info]   at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
[info]   at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.convertibleFilters(OrcFilters.scala:145)
[info]   at org.apache.spark.sql.hive.orc.OrcFilters$.createFilter(OrcFilters.scala:83)
[info]   at org.apache.spark.sql.hive.orc.OrcFileFormat.buildReader(OrcFileFormat.scala:142)
```

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

No.

### How was this patch tested?

Unit test.

Closes #29526 from viirya/SPARK-32352-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-24 14:06:08 +09:00
mingjial b9585cde31 [SPARK-32609][TEST] Add Tests for Incorrect exchange reuse with DataSourceV2
### What changes were proposed in this pull request?
Copy  to master branch the unit test added for branch-2.4(https://github.com/apache/spark/pull/29430).

### Why are the changes needed?
The unit test will pass at master branch, indicating that issue reported in https://issues.apache.org/jira/browse/SPARK-32609 is already fixed at master branch. But adding this unit test for future possible failure catch.

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

### How was this patch tested?
sbt test run

Closes #29435 from mingjialiu/master.

Authored-by: mingjial <mingjial@google.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-23 17:40:59 -07:00
majsiel 772c70615a [SPARK-32648][SS] Remove unused DELETE_ACTION in FileStreamSinkLog
### What changes were proposed in this pull request?
Removing unused DELETE_ACTION in FileStreamSinkLog.

### Why are the changes needed?
DELETE_ACTION is not used nowhere in the code.

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

### How was this patch tested?
Tests where not added, because code was removed.

Closes #29505 from michal-wieleba/SPARK-32648.

Authored-by: majsiel <majsiel@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-23 17:17:39 -07:00
Max Gekk 1c798f973f [SPARK-32594][SQL][FOLLOWUP][TEST-HADOOP2.7][TEST-HIVE1.2] Override get() and use Julian days in DaysWritable
### What changes were proposed in this pull request?
Override `def get: Date` in `DaysWritable` use the `daysToMillis(int d)` from the parent class `DateWritable` instead of `long daysToMillis(int d, boolean doesTimeMatter)`.

### Why are the changes needed?
It fixes failures of `HiveSerDeReadWriteSuite` with the profile `hive-1.2`. In that case, the parent class `DateWritable` has different implementation before the commit to Hive da3ed68eda. In particular, `get()` calls `new Date(daysToMillis(daysSinceEpoch))` instead of overrided `def get(doesTimeMatter: Boolean): Date` in the child class. The `get()` method returns wrong result `1970-01-01` because it uses not updated `daysSinceEpoch`.

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

### How was this patch tested?
By running the test suite `HiveSerDeReadWriteSuite`:
```
$ build/sbt -Phive-1.2 -Phadoop-2.7 "test:testOnly org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite"
```
and
```
$ build/sbt -Phive-2.3 -Phadoop-2.7 "test:testOnly org.apache.spark.sql.hive.execution.HiveSerDeReadWriteSuite"
```

Closes #29523 from MaxGekk/insert-date-into-hive-table-1.2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 12:43:30 -07:00
angerszhu 9808c15eec [SPARK-32608][SQL][FOLLOW-UP][TEST-HADOOP2.7][TEST-HIVE1.2] Script Transform ROW FORMAT DELIMIT value should format value
### What changes were proposed in this pull request?
As mentioned in https://github.com/apache/spark/pull/29428#issuecomment-678735163 by viirya ,
fix bug in UT, since in script transformation no-serde mode, output of decimal is same in both hive-1.2/hive-2.3

### Why are the changes needed?
FIX UT

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

### How was this patch tested?
EXISTED UT

Closes #29520 from AngersZhuuuu/SPARK-32608-FOLLOW.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 08:08:55 -07:00
Liang-Chi Hsieh aa0b0b87e2 Revert "[SPARK-32646][SQL] ORC predicate pushdown should work with case-insensitive analysis"
### What changes were proposed in this pull request?

This reverts commit e277ef1a83.

### Why are the changes needed?

Because master and branch-3.0 both have few tests failed under hive-1.2 profile. And the PR #29457 missed a change in hive-1.2 code that causes compilation error. So it will make debugging the failed tests harder. I'd like revert #29457 first to unblock it.

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

No

### How was this patch tested?

Unit test

Closes #29519 from viirya/revert-SPARK-32646.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-23 00:40:53 -07:00
xuewei.linxuewei f258718535 [SPARK-32678][SQL] Rename EmptyHashedRelationWithAllNullKeys and simplify NAAJ generated code
### What changes were proposed in this pull request?
Some Code refine.

1. rename EmptyHashedRelationWithAllNullKeys to HashedRelationWithAllNullKeys.
2. simplify generated code for BHJ NAAJ.

### Why are the changes needed?
Refine code and naming to avoid confusing understanding.

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

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

Closes #29503 from leanken/leanken-SPARK-32678.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2020-08-22 22:32:39 -07:00
yangjie01 25c7d0fe6a [SPARK-32526][SQL] Pass all test of sql/catalyst module in Scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to resolve [SPARK-32526](https://issues.apache.org/jira/browse/SPARK-32526), all remaining failed cases are fixed.

The main change of this pr as follow:

- Change of `ExecutorAllocationManager.scala` for core module compilation in Scala 2.13, it's a blocking problem

- Change `Seq[_]` to `scala.collection.Seq[_]` refer to failed cases

- Added different expected plan of `Test 4: Star with several branches` of StarJoinCostBasedReorderSuite  for Scala 2.13 because the candidates plans:

```
Join Inner, (d1_pk#5 = f1_fk1#0)
:- Join Inner, (f1_fk2#1 = d2_pk#8)
:  :- Join Inner, (f1_fk3#2 = d3_pk#11)
```
and

```
Join Inner, (f1_fk2#1 = d2_pk#8)
:- Join Inner, (d1_pk#5 = f1_fk1#0)
:  :- Join Inner, (f1_fk3#2 = d3_pk#11)
```

have same cost `Cost(200,9200)`, but `HashMap` is rewritten in scala 2.13 and The order of iterations leads to different results.

This pr fix test cases as follow:

- LiteralExpressionSuite (1 FAILED -> PASS)
- StarJoinCostBasedReorderSuite ( 1 FAILED-> PASS)
- ObjectExpressionsSuite( 2 FAILED-> PASS)
- ScalaReflectionSuite (1 FAILED-> PASS)
- RowEncoderSuite (10 FAILED-> PASS)
- ExpressionEncoderSuite  (ABORTED-> PASS)

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?
<!--
- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/catalyst -Pscala-2.13 -am
mvn test -pl sql/catalyst -Pscala-2.13
```

**Before**
```
Tests: succeeded 4035, failed 17, canceled 0, ignored 6, pending 0
*** 1 SUITE ABORTED ***
*** 15 TESTS FAILED ***
```

**After**

```
Tests: succeeded 4338, failed 0, canceled 0, ignored 6, pending 0
All tests passed.
```

Closes #29434 from LuciferYang/sql-catalyst-tests.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-08-22 09:24:16 -05:00
Yuanjian Li 8b26c69ce7 [SPARK-31792][SS][DOC][FOLLOW-UP] Rephrase the description for some operations
### What changes were proposed in this pull request?
Rephrase the description for some operations to make it clearer.

### Why are the changes needed?
Add more detail in the document.

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

### How was this patch tested?
Document only.

Closes #29269 from xuanyuanking/SPARK-31792-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-08-22 21:32:23 +09:00
Robert (Bobby) Evans 12f4331b9e [SPARK-32672][SQL] Fix data corruption in boolean bit set compression
## What changes were proposed in this pull request?

This fixed SPARK-32672 a data corruption.  Essentially the BooleanBitSet CompressionScheme would miss nulls at the end of a CompressedBatch.  The values would then default to false.

### Why are the changes needed?
It fixes data corruption

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

### How was this patch tested?
I manually tested it against the original issue that was producing errors for me.  I also added in a unit test.

Closes #29506 from revans2/SPARK-32672.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-22 11:07:14 +09:00
Brandon Jiang 1450b5e095 [MINOR][DOCS] fix typo for docs,log message and comments
### What changes were proposed in this pull request?
Fix typo for docs, log messages and comments

### Why are the changes needed?
typo fix to increase readability

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

### How was this patch tested?
manual test has been performed to test the updated

Closes #29443 from brandonJY/spell-fix-doc.

Authored-by: Brandon Jiang <Brandon.jiang.a@outlook.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:45:35 +09:00
Wenchen Fan 3dca81e4f5 [SPARK-32669][SQL][TEST] Expression unit tests should explore all cases that can lead to null result
### What changes were proposed in this pull request?

Add document to `ExpressionEvalHelper`, and ask people to explore all the cases that can lead to null results (including null in struct fields, array elements and map values).

This PR also fixes `ComplexTypeSuite.GetArrayStructFields` to explore all the null cases.

### Why are the changes needed?

It happened several times that we hit correctness bugs caused by wrong expression nullability. When writing unit tests, we usually don't test the nullability flag directly, and it's too late to add such tests for all expressions.

In https://github.com/apache/spark/pull/22375, we extended the expression test framework, which checks the nullability flag when the expected result/field/element is null.

This requires the test cases to explore all the cases that can lead to null results

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

no

### How was this patch tested?

I reverted 5d296ed39e locally, and `ComplexTypeSuite` can catch the bug.

Closes #29493 from cloud-fan/small.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-22 06:23:46 +09:00
Liang-Chi Hsieh e277ef1a83 [SPARK-32646][SQL] ORC predicate pushdown should work with case-insensitive analysis
### What changes were proposed in this pull request?

This PR proposes to fix ORC predicate pushdown under case-insensitive analysis case. The field names in pushed down predicates don't need to match in exact letter case with physical field names in ORC files, if we enable case-insensitive analysis.

### Why are the changes needed?

Currently ORC predicate pushdown doesn't work with case-insensitive analysis. A predicate "a < 0" cannot pushdown to ORC file with field name "A" under case-insensitive analysis.

But Parquet predicate pushdown works with this case. We should make ORC predicate pushdown work with case-insensitive analysis too.

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

Yes, after this PR, under case-insensitive analysis, ORC predicate pushdown will work.

### How was this patch tested?

Unit tests.

Closes #29457 from viirya/fix-orc-pushdown.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-21 07:57:24 +00:00
angerszhu c75a82794f [SPARK-32667][SQL] Script transform 'default-serde' mode should pad null value to filling column
### What changes were proposed in this pull request?
Hive no serde mode when  column less then output specified column, it will pad null value to it, spark should do this also.
```
hive> SELECT TRANSFORM(a, b)
    >   ROW FORMAT DELIMITED
    >   FIELDS TERMINATED BY '|'
    >   LINES TERMINATED BY '\n'
    >   NULL DEFINED AS 'NULL'
    > USING 'cat' as (a string, b string, c string, d string)
    >   ROW FORMAT DELIMITED
    >   FIELDS TERMINATED BY '|'
    >   LINES TERMINATED BY '\n'
    >   NULL DEFINED AS 'NULL'
    > FROM (
    > select 1 as a, 2 as b
    > ) tmp ;
OK
1	2	NULL	NULL
Time taken: 24.626 seconds, Fetched: 1 row(s)
```

### Why are the changes needed?
Keep save behavior with hive data.

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

### How was this patch tested?
Added UT

Closes #29500 from AngersZhuuuu/SPARK-32667.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-21 07:37:11 +00:00
Wenchen Fan 8b119f1663 [SPARK-32640][SQL] Downgrade Janino to fix a correctness bug
### What changes were proposed in this pull request?

This PR reverts https://github.com/apache/spark/pull/27860 to downgrade Janino, as the new version has a bug.

### Why are the changes needed?

The symptom is about NaN comparison. For code below
```
if (double_value <= 0.0) {
  ...
} else {
  ...
}
```

If `double_value` is NaN, `NaN <= 0.0` is false and we should go to the else branch. However, current Spark goes to the if branch and causes correctness issues like SPARK-32640.

One way to fix it is:
```
boolean cond = double_value <= 0.0;
if (cond) {
  ...
} else {
  ...
}
```

I'm not familiar with Janino so I don't know what's going on there.

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

Yes, fix correctness bugs.

### How was this patch tested?

a new test

Closes #29495 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-20 13:26:39 -07:00
Wenchen Fan d378dc5f6d [SPARK-28863][SQL][FOLLOWUP] Do not reuse the physical plan
### What changes were proposed in this pull request?

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

Instead of passing the physical plan to the fallbacked v1 source directly and skipping analysis, optimization, planning altogether, this PR proposes to pass the optimized plan.

### Why are the changes needed?

It's a bit risky to pass the physical plan directly. When the fallbacked v1 source applies more operations to the input DataFrame, it will re-apply the post-planning physical rules like `CollapseCodegenStages`, `InsertAdaptiveSparkPlan`, etc., which is very tricky.

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

No

### How was this patch tested?

existing test suite with some new tests

Closes #29489 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-20 15:23:25 +00:00
Takeshi Yamamuro d80d0ced9a [SPARK-32665][SQL][TEST] Deletes orphan directories under a warehouse dir in SQLQueryTestSuite
### What changes were proposed in this pull request?

In case that a last `SQLQueryTestSuite` test run is killed, it will fail in a next run because of a following reason:
```
[info] org.apache.spark.sql.SQLQueryTestSuite *** ABORTED *** (17 seconds, 483 milliseconds)
[info]   org.apache.spark.sql.AnalysisException: Can not create the managed table('`testdata`'). The associated location('file:/Users/maropu/Repositories/spark/spark-master/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/testdata') already exists.;
[info]   at org.apache.spark.sql.catalyst.catalog.SessionCatalog.validateTableLocation(SessionCatalog.scala:355)
[info]   at org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand.run(createDataSourceTables.scala:170)
[info]   at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108)
```
This PR intends to add code to deletes orphan directories under a warehouse dir in `SQLQueryTestSuite` before creating test tables.

### Why are the changes needed?

To improve test convenience

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

No

### How was this patch tested?

Manually checked

Closes #29488 from maropu/DeleteDirs.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-20 06:12:05 -07:00
angerszhu 6dae11d034 [SPARK-32607][SQL] Script Transformation ROW FORMAT DELIMITED TOK_TABLEROWFORMATLINES only support '\n'
### What changes were proposed in this pull request?
Scrip Transform no-serde (`ROW FORMAT DELIMITED`) mode `LINE TERMINNATED BY `
only support `\n`.

Tested in hive :
Hive 1.1
![image](https://user-images.githubusercontent.com/46485123/90309510-ce82a180-df1b-11ea-96ab-56e2b3229489.png)

Hive 2.3.7
![image](https://user-images.githubusercontent.com/46485123/90309504-c88cc080-df1b-11ea-853e-8f65e9ed2375.png)

### Why are the changes needed?
Strictly limit the use method to ensure the accuracy of data

### Does this PR introduce _any_ user-facing change?
User use Scrip Transform no-serde (ROW FORMAT DELIMITED) mode  with `LINE TERMINNATED BY `
not equal `'\n'`. will throw error

### How was this patch tested?
Added UT

Closes #29438 from AngersZhuuuu/SPARK-32607.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-20 12:03:02 +00:00
Jungtaek Lim (HeartSaVioR) e6795cd341 [SPARK-30462][SS] Streamline the logic on file stream source and sink metadata log to avoid memory issue
### What changes were proposed in this pull request?

In many operations on CompactibleFileStreamLog reads a metadata log file and materializes all entries into memory. As the nature of the compact operation, CompactibleFileStreamLog may have a huge compact log file with bunch of entries included, and for now they're just monotonically increasing, which means the amount of memory to materialize also grows incrementally. This leads pressure on GC.

This patch proposes to streamline the logic on file stream source and sink whenever possible to avoid memory issue. To make this possible we have to break the existing behavior of excluding entries - now the `compactLogs` method is called with all entries, which forces us to materialize all entries into memory. This is hopefully no effect on end users, because only file stream sink has a condition to exclude entries, and the condition has been never true. (DELETE_ACTION has been never set.)

Based on the observation, this patch also changes the existing UT a bit which simulates the situation where "A" file is added, and another batch marks the "A" file as deleted. This situation simply doesn't work with the change, but as I mentioned earlier it hasn't been used. (I'm not sure the UT is from the actual run. I guess not.)

### Why are the changes needed?

The memory issue (OOME) is reported by both JIRA issue and user mailing list.

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

No.

### How was this patch tested?

* Existing UTs
* Manual test done

The manual test leverages the simple apps which continuously writes the file stream sink metadata log.

bea7680e4c

The test is configured to have a batch metadata log file at 1.9M (10,000 entries) whereas other Spark configuration is set to the default. (compact interval = 10) The app runs as driver, and the heap memory on driver is set to 3g.

> before the patch

<img width="1094" alt="Screen Shot 2020-06-23 at 3 37 44 PM" src="https://user-images.githubusercontent.com/1317309/85375841-d94f3480-b571-11ea-817b-c6b48b34888a.png">

It only ran for 40 mins, with the latest compact batch file size as 1.3G. The process struggled with GC, and after some struggling, it threw OOME.

> after the patch

<img width="1094" alt="Screen Shot 2020-06-23 at 3 53 29 PM" src="https://user-images.githubusercontent.com/1317309/85375901-eff58b80-b571-11ea-837e-30d107f677f9.png">

It sustained 2 hours run (manually stopped as it's expected to run more), with the latest compact batch file size as 2.2G. The actual memory usage didn't even go up to 1.2G, and be cleaned up soon without outstanding GC activity.

Closes #28904 from HeartSaVioR/SPARK-30462.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-20 02:26:38 -07:00
ulysses 7048fff230 [SPARK-31999][SQL][FOLLOWUP] Adds negative test cases with typos
### What changes were proposed in this pull request?

Address the [#comment](https://github.com/apache/spark/pull/28840#discussion_r471172006).

### Why are the changes needed?

Make code robust.

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

No.

### How was this patch tested?

ut.

Closes #29453 from ulysses-you/SPARK-31999-FOLLOWUP.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-20 05:33:12 +00:00
Burak Yavuz 278d0dd25b [SPARK-28863][SQL] Introduce AlreadyPlanned to prevent reanalysis of V1FallbackWriters
### What changes were proposed in this pull request?

This PR introduces a LogicalNode AlreadyPlanned, and related physical plan and preparation rule.

With the DataSourceV2 write operations, we have a way to fallback to the V1 writer APIs using InsertableRelation. The gross part is that we're in physical land, but the InsertableRelation takes a logical plan, so we have to pass the logical plans to these physical nodes, and then potentially go through re-planning. This re-planning can cause issues for an already optimized plan.

A useful primitive could be specifying that a plan is ready for execution through a logical node AlreadyPlanned. This would wrap a physical plan, and then we can go straight to execution.

### Why are the changes needed?

To avoid having a physical plan that is disconnected from the physical plan that is being executed in V1WriteFallback execution. When a physical plan node executes a logical plan, the inner query is not connected to the running physical plan. The physical plan that actually runs is not visible through the Spark UI and its metrics are not exposed. In some cases, the EXPLAIN plan doesn't show it.

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

Nope

### How was this patch tested?

V1FallbackWriterSuite tests that writes still work

Closes #29469 from brkyvz/alreadyAnalyzed2.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 16:25:35 +00:00
Terry Kim 3d1dce75d9 [SPARK-32621][SQL] 'path' option can cause issues while inferring schema in CSV/JSON datasources
### What changes were proposed in this pull request?

When CSV/JSON datasources infer schema (e.g, `def inferSchema(files: Seq[FileStatus])`, they use the `files` along with the original options. `files` in `inferSchema` could have been deduced from the "path" option if the option was present, so this can cause issues (e.g., reading more data, listing the path again) since the "path" option is **added** to the `files`.

### Why are the changes needed?

The current behavior can cause the following issue:
```scala
class TestFileFilter extends PathFilter {
  override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
}

val path = "/tmp"
val df = spark.range(2)
df.write.json(path + "/p=1")
df.write.json(path + "/p=2")

val extraOptions = Map(
  "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName,
  "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName
)

// This works fine.
assert(spark.read.options(extraOptions).json(path).count == 2)

// The following with "path" option fails with the following:
// assertion failed: Conflicting directory structures detected. Suspicious paths
//	file:/tmp
//	file:/tmp/p=1
assert(spark.read.options(extraOptions).format("json").option("path", path).load.count() === 2)
```

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

Yes, the above failure doesn't happen and you get the consistent experience when you use `spark.read.csv(path)` or `spark.read.format("csv").option("path", path).load`.

### How was this patch tested?

Updated existing tests.

Closes #29437 from imback82/path_bug.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 16:23:22 +00:00
Yuming Wang 409fea30cc [SPARK-32624][SQL] Use getCanonicalName to fix byte[] compile issue
### What changes were proposed in this pull request?
```scala
scala> Array[Byte](1, 2).getClass.getName
res13: String = [B

scala> Array[Byte](1, 2).getClass.getCanonicalName
res14: String = byte[]
```

This pr replace `getClass.getName` with `getClass.getCanonicalName` in `CodegenContext.addReferenceObj` to fix `byte[]` compile issue:
```
...
/* 030 */       value_1 = org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary(value_2, (([B) references[0] /* min */)) >= 0 && org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary(value_2, (([B) references[1] /* max */)) <= 0;
/* 031 */     }
/* 032 */     return !isNull_1 && value_1;
/* 033 */   }
/* 034 */
/* 035 */
/* 036 */ }

20:49:54.886 WARN org.apache.spark.sql.catalyst.expressions.Predicate: Expr codegen error and falling back to interpreter mode
java.util.concurrent.ExecutionException: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 30, Column 81: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 30, Column 81: Unexpected token "[" in primary
...
```

### Why are the changes needed?

Fix compile issue when compiling generated code.

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

No.

### How was this patch tested?

Unit test.

Closes #29439 from wangyum/SPARK-32624.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-08-19 05:20:26 -07:00
angerszhu 03e2de99ab [SPARK-32608][SQL] Script Transform ROW FORMAT DELIMIT value should format value
### What changes were proposed in this pull request?
For SQL
```
SELECT TRANSFORM(a, b, c)
  ROW FORMAT DELIMITED
  FIELDS TERMINATED BY ','
  LINES TERMINATED BY '\n'
  NULL DEFINED AS 'null'
  USING 'cat' AS (a, b, c)
  ROW FORMAT DELIMITED
  FIELDS TERMINATED BY ','
  LINES TERMINATED BY '\n'
  NULL DEFINED AS 'NULL'
FROM testData
```
The correct

TOK_TABLEROWFORMATFIELD should be `, `nut actually ` ','`

TOK_TABLEROWFORMATLINES should be `\n`  but actually` '\n'`

### Why are the changes needed?
Fix string value format

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

### How was this patch tested?
Added UT

Closes #29428 from AngersZhuuuu/SPARK-32608.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-19 08:31:58 +00:00
Samir Khan e15ae60a53 [SPARK-32550][SQL] Make SpecificInternalRow constructors faster by using while loops instead of maps
### What changes were proposed in this pull request?
Change maps in two constructors of SpecificInternalRow to while loops.

### Why are the changes needed?
This was originally noticed with https://github.com/apache/spark/pull/29353 and https://github.com/apache/spark/pull/29354 and will have impacts on performance of reading ORC and Avro files. Ran AvroReadBenchmarks with the new cases of nested and array'd structs in https://github.com/apache/spark/pull/29352. Haven't run benchmarks for ORC but can do that if needed.

**Before:**
```
Nested Struct Scan:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Nested Struct                                     74674          75319         912          0.0      142429.1       1.0X

Array of Struct Scan:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Array of Structs                                  34193          34339         206          0.0       65217.9       1.0X
```
**After:**
```
Nested Struct Scan:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Nested Struct                                     48451          48619         237          0.0       92413.2       1.0X

Array of Struct Scan:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
Array of Structs                                  18518          18683         234          0.0       35319.6       1.0X
```

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

### How was this patch tested?
Ran AvroReadBenchmarks with the new cases of nested and array'd structs in https://github.com/apache/spark/pull/29352.

Closes #29366 from msamirkhan/spark-32550.

Lead-authored-by: Samir Khan <muhammad.samir.khan@gmail.com>
Co-authored-by: skhan04 <samirkhan@verizonmedia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-19 14:57:34 +09:00
Wenchen Fan f33b64a656 [SPARK-32652][SQL] ObjectSerializerPruning fails for RowEncoder
### What changes were proposed in this pull request?

Update `ObjectSerializerPruning.alignNullTypeInIf`, to consider the isNull check generated in `RowEncoder`, which is `Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil)`.

### Why are the changes needed?

Query fails if we don't fix this bug, due to type mismatch in `If`.

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

Yes, the failed query can run after this fix.

### How was this patch tested?

new tests

Closes #29467 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-19 13:50:29 +09:00
Liang-Chi Hsieh b33066f42b [SPARK-32622][SQL][TEST] Add case-sensitivity test for ORC predicate pushdown
### What changes were proposed in this pull request?

During working on SPARK-25557, we found that ORC predicate pushdown doesn't have case-sensitivity test. This PR proposes to add case-sensitivity test for ORC predicate pushdown.

### Why are the changes needed?

Increasing test coverage for ORC predicate pushdown.

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

No

### How was this patch tested?

Pass Jenkins tests.

Closes #29427 from viirya/SPARK-25557-followup3.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-17 13:19:49 -07:00
xuewei.linxuewei 108b1dc723 [SPARK-32615][SQL] Fix AQE aggregateMetrics java.util.NoSuchElementException
### What changes were proposed in this pull request?
Found java.util.NoSuchElementException in UT log of AdaptiveQueryExecSuite. During AQE, when sub-plan changed, LiveExecutionData is using the new sub-plan SQLMetrics to override the old ones, But in the final aggregateMetrics, since the plan was updated, the old metrics will throw NoSuchElementException when it try to match with the new metricTypes. To sum up, we need to filter out those outdated metrics to avoid throwing java.util.NoSuchElementException,  which cause SparkUI SQL Tab abnormally rendered.

### Why are the changes needed?
SQL Metrics is not correct for some AQE cases, and it break SparkUI SQL Tab when it comes to NAAJ rewritten to LocalRelation case.

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

### How was this patch tested?
* Added case in SQLAppStatusListenerSuite.
* Run AdaptiveQueryExecSuite with no "java.util.NoSuchElementException".
* Validation on Spark Web UI

Closes #29431 from leanken/leanken-SPARK-32615.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-17 14:28:10 +00:00
yi.wu 9f2893cf2c [SPARK-32466][TEST][SQL] Add PlanStabilitySuite to detect SparkPlan regression
### What changes were proposed in this pull request?

This PR proposes to detect possible regression inside `SparkPlan`. To achieve this goal, this PR added a base test suite called  `PlanStabilitySuite`. The basic workflow of this test suite is similar to  `SQLQueryTestSuite`. It also uses `SPARK_GENERATE_GOLDEN_FILES` to decide whether it should regenerate the golden files or compare to the golden result for each input query. The difference is, `PlanStabilitySuite` uses the serialized explain result(.txt format) of the `SparkPlan` as the output of a query, instead of the data result.

And since `SparkPlan` is non-deterministic for various reasons, e.g.,  expressions ids changes, expression order changes, we'd reduce the plan to a simplified version that only contains node names and references. And we only identify those important nodes, e.g., `Exchange`, `SubqueryExec`, in the simplified plan.

And we'd reuse TPC-DS queries(v1.4, v2.7, modified) to test plans' stability. Currently, one TPC-DS query can only have one corresponding simplified golden plan.

This PR also did a few refactor, which extracts `TPCDSBase` from `TPCDSQuerySuite`. So,  `PlanStabilitySuite` can use the TPC-DS queries as well.

### Why are the changes needed?

Nowadays, Spark is getting more and more complex. Any changes might cause regression unintentionally. Spark already has some benchmark to catch the performance regression. But, yet, it doesn't have a way to detect the regression inside `SparkPlan`. It would be good if we could detect the possible regression early during the compile phase before the runtime phase.

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

No.

### How was this patch tested?

Added `PlanStabilitySuite` and it's subclasses.

Closes #29270 from Ngone51/plan-stable.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-17 14:22:12 +00:00
Wenchen Fan b94c67b502 Revert "[SPARK-32511][SQL] Add dropFields method to Column class"
This reverts commit 0c850c71e7.
2020-08-17 13:18:46 +08:00
Cheng Su 8f0fef1843 [SPARK-32399][SQL] Full outer shuffled hash join
### What changes were proposed in this pull request?

Add support for full outer join inside shuffled hash join. Currently if the query is a full outer join, we only use sort merge join as the physical operator. However it can be CPU and IO intensive in case input table is large for sort merge join. Shuffled hash join on the other hand saves the sort CPU and IO compared to sort merge join, especially when table is large.

This PR implements the full outer join as followed:
* Process rows from stream side by looking up hash relation, and mark the matched rows from build side by:
  * for joining with unique key, a `BitSet` is used to record matched rows from build side (`key index` to represent each row)
  * for joining with non-unique key, a `HashSet[Long]` is  used to record matched rows from build side (`key index` + `value index` to represent each row).
`key index` is defined as the index into key addressing array `longArray` in `BytesToBytesMap`.
`value index` is defined as the iterator index of values for same key.

* Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`)

For context, this PR was originally implemented as followed (up to commit e3322766d4):
1. Construct hash relation from build side, with extra boolean value at the end of row to track look up information (done in `ShuffledHashJoinExec.buildHashedRelation` and `UnsafeHashedRelation.apply`).
2. Process rows from stream side by looking up hash relation, and mark the matched rows from build side be looked up (done in `ShuffledHashJoinExec.fullOuterJoin`).
3. Process rows from build side by iterating hash relation, and filter out rows from build side being looked up already (done in `ShuffledHashJoinExec.fullOuterJoin`).

See discussion of pros and cons between these two approaches [here](https://github.com/apache/spark/pull/29342#issuecomment-672275450), [here](https://github.com/apache/spark/pull/29342#issuecomment-672288194) and [here](https://github.com/apache/spark/pull/29342#issuecomment-672640531).

TODO: codegen for full outer shuffled hash join can be implemented in another followup PR.

### Why are the changes needed?

As implementation in this PR, full outer shuffled hash join will have overhead to iterate build side twice (once for building hash map, and another for outputting non-matching rows), and iterate stream side once. However, full outer sort merge join needs to iterate both sides twice, and sort the large table can be more CPU and IO intensive. So full outer shuffled hash join can be more efficient than sort merge join when stream side is much more larger than build side.

For example query below, full outer SHJ saved 30% wall clock time compared to full outer SMJ.

```
def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 22
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"cast(id as string) as k1")
        val df2 = spark.range(N / 10).selectExpr(s"cast(id * 10 as string) as k2")
        val df = df1.join(df2, col("k1") === col("k2"), "full_outer")
        df.noop()
    }
  }
}
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join off
  Stopped after 2 iterations, 16602 ms
  Running case: shuffle hash join on
  Stopped after 5 iterations, 31911 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join off                              7900           8301         567          2.1         470.9       1.0X
shuffle hash join on                               6250           6382          95          2.7         372.5       1.3X
```

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

No.

### How was this patch tested?

Added unit test in `JoinSuite.scala`, `AbstractBytesToBytesMapSuite.java` and `HashedRelationSuite.scala`.

Closes #29342 from c21/full-outer-shj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-17 08:06:19 +09:00
Yuming Wang c280c7f529 [SPARK-32625][SQL] Log error message when falling back to interpreter mode
### What changes were proposed in this pull request?

This pr log the error message when falling back to interpreter mode.

### Why are the changes needed?

Not all error messages are in `CodeGenerator`, such as:
```
21:48:44.612 WARN org.apache.spark.sql.catalyst.expressions.Predicate: Expr codegen error and falling back to interpreter mode
java.lang.IllegalArgumentException: Can not interpolate org.apache.spark.sql.types.Decimal into code block.
	at org.apache.spark.sql.catalyst.expressions.codegen.Block$BlockHelper$.$anonfun$code$1(javaCode.scala:240)
	at org.apache.spark.sql.catalyst.expressions.codegen.Block$BlockHelper$.$anonfun$code$1$adapted(javaCode.scala:236)
	at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
	at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
```

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

No.

### How was this patch tested?

Manual test.

Closes #29440 from wangyum/SPARK-32625.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-15 12:31:32 -07:00
yi.wu c6be2074cc [SPARK-32616][SQL] Window operators should be added determinedly
### What changes were proposed in this pull request?

Use the `LinkedHashMap` instead of `immutable.Map` to hold the `Window` expressions in `ExtractWindowExpressions.addWindow`.

### Why are the changes needed?

This is a bug fix for https://github.com/apache/spark/pull/29270. In that PR, the generated plan(especially for the queries q47, q49, q57) on Jenkins always can not match the golden plan generated on my laptop.

It happens because `ExtractWindowExpressions.addWindow` now uses `immutable.Map` to hold the `Window` expressions by the key `(spec.partitionSpec, spec.orderSpec, WindowFunctionType.functionType(expr))` and converts the map to `Seq` at the end. Then, the `Seq` is used to add Window operators on top of the child plan. However, for the same query, the order of Windows expression inside the `Seq` could be undetermined when the expression id changes(which can affect the key). As a result, the same query could have different plans because of the undetermined order of Window operators.

Therefore, we use `LinkedHashMap`, which records the insertion order of entries, to make the adding order determined.

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

Maybe yes, users now always see the same plan for the same queries with multiple Window operators.

### How was this patch tested?

It's really hard to make a reproduce demo. I just tested manually with https://github.com/apache/spark/pull/29270 and it looks good.

Closes #29432 from Ngone51/fix-addWindow.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-14 13:29:48 +00:00
alexander-daskalov 10edeafc69 [MINOR][SQL] Fixed approx_count_distinct rsd param description
### What changes were proposed in this pull request?

In the docs concerning the approx_count_distinct I have changed the description of the rsd parameter from **_maximum estimation error allowed_** to _**maximum relative standard deviation allowed**_

### Why are the changes needed?

Maximum estimation error allowed can be misleading. You can set the target relative standard deviation, which affects the estimation error, but on given runs the estimation error can still be above the rsd parameter.

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

This PR should make it easier for users reading the docs to understand that the rsd parameter in approx_count_distinct doesn't cap the estimation error, but just sets the target deviation instead,

### How was this patch tested?

No tests, as no code changes were made.

Closes #29424 from Comonut/fix-approx_count_distinct-rsd-param-description.

Authored-by: alexander-daskalov <alexander.daskalov@adevinta.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-08-14 22:10:41 +09:00
Huaxin Gao 14003d4c30 [SPARK-32590][SQL] Remove fullOutput from RowDataSourceScanExec
### What changes were proposed in this pull request?
Remove `fullOutput` from `RowDataSourceScanExec`

### Why are the changes needed?
`RowDataSourceScanExec` requires the full output instead of the scan output after column pruning. However, in v2 code path, we don't have the full output anymore so we just pass the pruned output. `RowDataSourceScanExec.fullOutput` is actually meaningless so we should remove it.

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

### How was this patch tested?
existing tests

Closes #29415 from huaxingao/rm_full_output.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-14 08:08:16 +00:00
ulysses 339eec5f32 [SPARK-20680][SQL][FOLLOW-UP] Add HiveVoidType in HiveClientImpl
### What changes were proposed in this pull request?

Discussion with [comment](https://github.com/apache/spark/pull/29244#issuecomment-671746329).

Add `HiveVoidType` class in `HiveClientImpl` then we can replace `NullType` to `HiveVoidType` before we call hive client.

### Why are the changes needed?

Better compatible with hive.

More details in [#29244](https://github.com/apache/spark/pull/29244).

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

Yes, user can create view with null type in Hive.

### How was this patch tested?

New test.

Closes #29423 from ulysses-you/add-HiveVoidType.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-14 06:59:15 +00:00
yangjie01 6ae2cb2db3 [SPARK-32526][SQL] Fix some test cases of sql/catalyst module in scala 2.13
### What changes were proposed in this pull request?
The purpose of this pr is to partial resolve [SPARK-32526](https://issues.apache.org/jira/browse/SPARK-32526), total of 88 failed and 2 aborted test cases were fixed, the related suite as follow:

- `DataSourceV2AnalysisBaseSuite` related test cases (71 FAILED -> Pass)
- `TreeNodeSuite` (1 FAILED -> Pass)
- `MetadataSuite `(1 FAILED -> Pass)
- `InferFiltersFromConstraintsSuite `(3 FAILED -> Pass)
- `StringExpressionsSuite ` (1 FAILED -> Pass)
- `JacksonParserSuite ` (1 FAILED -> Pass)
- `HigherOrderFunctionsSuite `(1 FAILED -> Pass)
- `ExpressionParserSuite` (1 FAILED -> Pass)
- `CollectionExpressionsSuite `(6 FAILED -> Pass)
- `SchemaUtilsSuite` (2 FAILED -> Pass)
- `ExpressionSetSuite `(ABORTED -> Pass)
- `ArrayDataIndexedSeqSuite `(ABORTED -> Pass)

The main change of this pr as following:

- `Optimizer` and `Analyzer` are changed to pass compile, `ArrayBuffer` is not a `Seq` in scala 2.13, call `toSeq` method manually to compatible with Scala 2.12

- `m.mapValues().view.force` pattern return a `Map` in scala 2.12 but return a `IndexedSeq` in scala 2.13, call `toMap` method manually to compatible with Scala 2.12. `TreeNode` are changed to pass `DataSourceV2AnalysisBaseSuite` related test cases and `TreeNodeSuite` failed case.

- call `toMap` method of `Metadata#hash` method `case map` branch because `map.mapValues` return `Map` in Scala 2.12 and return `MapView` in Scala 2.13.

- `impl` contact method of `ExpressionSet` in Scala 2.13 version refer to `ExpressionSet` in Scala 2.12 to support `+ + ` method conform to `ExpressionSet` semantics

- `GenericArrayData` not accept `ArrayBuffer` input, call `toSeq` when use `ArrayBuffer` construction `GenericArrayData`   for Scala version compatibility

-  Call `toSeq` in `RandomDataGenerator#randomRow` method to ensure contents of `fields` is `Seq` not `ArrayBuffer`

-  Call `toSeq` Let `JacksonParser#parse` still return a `Seq` because the check method of `JacksonParserSuite#"skipping rows using pushdown filters"` dependence on `Seq` type
- Call `toSeq` in `AstBuilder#visitFunctionCall`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13

- Add a `LongType` match to `ArraySetLike.nullValueHolder`

- Add a `sorted` to ensure `duplicateColumns` string in `SchemaUtils.checkColumnNameDuplication` method error message have a deterministic order

### Why are the changes needed?
We need to support a Scala 2.13 build.

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

### How was this patch tested?

- Scala 2.12: Pass the Jenkins or GitHub Action

- Scala 2.13: Do the following:

```
dev/change-scala-version.sh 2.13
mvn clean install -DskipTests  -pl sql/catalyst -Pscala-2.13 -am
mvn test -pl sql/catalyst -Pscala-2.13
```

**Before**
```
Tests: succeeded 3853, failed 103, canceled 0, ignored 6, pending 0
*** 3 SUITES ABORTED ***
*** 103 TESTS FAILED ***
```

**After**

```
Tests: succeeded 4035, failed 17, canceled 0, ignored 6, pending 0
*** 1 SUITE ABORTED ***
*** 15 TESTS FAILED ***
```

Closes #29370 from LuciferYang/fix-DataSourceV2AnalysisBaseSuite.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-08-13 11:46:30 -05:00
fqaiser94@gmail.com 0c850c71e7 [SPARK-32511][SQL] Add dropFields method to Column class
### What changes were proposed in this pull request?

Added a new `dropFields` method to the `Column` class.
This method should allow users to drop a `StructField` in a `StructType` column (with similar semantics to the `drop` method on `Dataset`).

### Why are the changes needed?

Often Spark users have to work with deeply nested data e.g. to fix a data quality issue with an existing `StructField`. To do this with the existing Spark APIs, users have to rebuild the entire struct column.

For example, let's say you have the following deeply nested data structure which has a data quality issue (`5` is missing):
```
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val data = spark.createDataFrame(sc.parallelize(
      Seq(Row(Row(Row(1, 2, 3), Row(Row(4, null, 6), Row(7, 8, 9), Row(10, 11, 12)), Row(13, 14, 15))))),
      StructType(Seq(
        StructField("a", StructType(Seq(
          StructField("a", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType)))),
          StructField("b", StructType(Seq(
            StructField("a", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("b", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType)))),
            StructField("c", StructType(Seq(
              StructField("a", IntegerType),
              StructField("b", IntegerType),
              StructField("c", IntegerType))))
          ))),
          StructField("c", StructType(Seq(
            StructField("a", IntegerType),
            StructField("b", IntegerType),
            StructField("c", IntegerType))))
        )))))).cache

data.show(false)
+---------------------------------+
|a                                |
+---------------------------------+
|[[1, 2, 3], [[4,, 6], [7, 8, 9]]]|
+---------------------------------+
```
Currently, to drop the missing value users would have to do something like this:
```
val result = data.withColumn("a",
  struct(
    $"a.a",
    struct(
      struct(
        $"a.b.a.a",
        $"a.b.a.c"
      ).as("a"),
      $"a.b.b",
      $"a.b.c"
    ).as("b"),
    $"a.c"
  ))

result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+
```
As you can see above, with the existing methods users must call the `struct` function and list all fields, including fields they don't want to change. This is not ideal as:
>this leads to complex, fragile code that cannot survive schema evolution.
[SPARK-16483](https://issues.apache.org/jira/browse/SPARK-16483)

In contrast, with the method added in this PR, a user could simply do something like this to get the same result:
```
val result = data.withColumn("a", 'a.dropFields("b.a.b"))
result.show(false)
+---------------------------------------------------------------+
|a                                                              |
+---------------------------------------------------------------+
|[[1, 2, 3], [[4, 6], [7, 8, 9], [10, 11, 12]], [13, 14, 15]]|
+---------------------------------------------------------------+

```

This is the second of maybe 3 methods that could be added to the `Column` class to make it easier to manipulate nested data.
Other methods under discussion in [SPARK-22231](https://issues.apache.org/jira/browse/SPARK-22231) include `withFieldRenamed`.
However, this should be added in a separate PR.

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

Only one minor change. If the user submits the following query:
```
df.withColumn("a", $"a".withField(null, null))
```
instead of throwing:
```
java.lang.IllegalArgumentException: requirement failed: fieldName cannot be null
```
it will now throw:
```
java.lang.IllegalArgumentException: requirement failed: col cannot be null
```
I don't believe its should be an issue to change this because:
- neither message is incorrect
- Spark 3.1.0 has yet to be released

but please feel free to correct me if I am wrong.

### How was this patch tested?

New unit tests were added. Jenkins must pass them.

### Related JIRAs:
More discussion on this topic can be found here:
- https://issues.apache.org/jira/browse/SPARK-22231
- https://issues.apache.org/jira/browse/SPARK-16483

Closes #29322 from fqaiser94/SPARK-32511.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-13 03:28:25 +00:00
Chen Zhang 08d86ebc05 [MINOR] Update URL of the parquet project in code comment
### What changes were proposed in this pull request?
Update URL of the parquet project in code comment.

### Why are the changes needed?
The original url is not available.

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

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

Closes #29416 from izchen/Update-Parquet-URL.

Authored-by: Chen Zhang <izchen@126.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-13 11:54:43 +09:00
stczwd 60fa8e304d [SPARK-31694][SQL] Add SupportsPartitions APIs on DataSourceV2
### What changes were proposed in this pull request?
There are no partition Commands, such as AlterTableAddPartition supported in DatasourceV2, it is widely used in mysql or hive or other datasources. Thus it is necessary to defined Partition API to support these Commands.

We defined the partition API as part of Table API, as it will change table data sometimes. And a partition is composed of identifier and properties, while identifier is defined with InternalRow and properties is defined as a Map.

### Does this PR introduce _any_ user-facing change?
Yes. This PR will enable user to use some partition commands

### How was this patch tested?
run all tests and add some partition api tests

Closes #28617 from stczwd/SPARK-31694.

Authored-by: stczwd <qcsd2011@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 17:25:47 +00:00
angerszhu 643cd876e4 [SPARK-32352][SQL] Partially push down support data filter if it mixed in partition filters
### What changes were proposed in this pull request?
We support partially push partition filters since SPARK-28169. We can also support partially push down data filters if it mixed in partition filters and data filters. For example:
```
spark.sql(
  s"""
     |CREATE TABLE t(i INT, p STRING)
     |USING parquet
     |PARTITIONED BY (p)""".stripMargin)

spark.range(0, 1000).selectExpr("id as col").createOrReplaceTempView("temp")
for (part <- Seq(1, 2, 3, 4)) {
  sql(s"""
         |INSERT OVERWRITE TABLE t PARTITION (p='$part')
         |SELECT col FROM temp""".stripMargin)
}

spark.sql("SELECT * FROM t WHERE  WHERE (p = '1' AND i = 1) OR (p = '2' and i = 2)").explain()
```

We can also push down ```i = 1 or i = 2 ```

### Why are the changes needed?
Extract more data filter to FileSourceScanExec

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

### How was this patch tested?
Added UT

Closes #29406 from AngersZhuuuu/SPARK-32352.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 12:18:33 +00:00
Max Gekk f664aaaab1 [SPARK-32599][SQL][TESTS] Check the TEXTFILE file format in HiveSerDeReadWriteSuite
### What changes were proposed in this pull request?
- Test TEXTFILE together with the PARQUET and ORC file formats in `HiveSerDeReadWriteSuite`
- Remove the "SPARK-32594: insert dates to a Hive table" added by #29409

### Why are the changes needed?
- To improve test coverage, and test other row SerDe - `org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe`.
- The removed test is not needed anymore because the bug reported in SPARK-32594 is triggered by the TEXTFILE file format too.

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

### How was this patch tested?
By running the modified test suite `HiveSerDeReadWriteSuite`.

Closes #29417 from MaxGekk/textfile-HiveSerDeReadWriteSuite.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 20:59:59 +09:00
Tin Hang To a418548dad [SPARK-31703][SQL] Parquet RLE float/double are read incorrectly on big endian platforms
### What changes were proposed in this pull request?
This PR fixes the issue introduced during SPARK-26985.

SPARK-26985 changes the `putDoubles()` and `putFloats()` methods to respect the platform's endian-ness.  However, that causes the RLE paths in VectorizedRleValuesReader.java to read the RLE entries in parquet as BIG_ENDIAN on big endian platforms (i.e., as is), even though parquet data is always in little endian format.

The comments in `WriteableColumnVector.java` say those methods are used for "ieee formatted doubles in platform native endian" (or floats), but since the data in parquet is always in little endian format, use of those methods appears to be inappropriate.

To demonstrate the problem with spark-shell:

```scala
import org.apache.spark._
import org.apache.spark.sql._
import org.apache.spark.sql.types._

var data = Seq(
  (1.0, 0.1),
  (2.0, 0.2),
  (0.3, 3.0),
  (4.0, 4.0),
  (5.0, 5.0))

var df = spark.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet("/tmp/data.parquet2")
var df2 = spark.read.parquet("/tmp/data.parquet2")
df2.show()
```

result:

```scala
+--------------------+--------------------+
|                  _1|                  _2|
+--------------------+--------------------+
|           3.16E-322|-1.54234871366845...|
|         2.0553E-320|         2.0553E-320|
|          2.561E-320|          2.561E-320|
|4.66726145843124E-62|         1.0435E-320|
|        3.03865E-319|-1.54234871366757...|
+--------------------+--------------------+
```

Also tests in ParquetIOSuite that involve float/double data would fail, e.g.,

- basic data types (without binary)
- read raw Parquet file

/examples/src/main/python/mllib/isotonic_regression_example.py would fail as well.

Purposed code change is to add `putDoublesLittleEndian()` and `putFloatsLittleEndian()` methods for parquet to invoke, just like the existing `putIntsLittleEndian()` and `putLongsLittleEndian()`.  On little endian platforms they would call `putDoubles()` and `putFloats()`, on big endian they would read the entries as little endian like pre-SPARK-26985.

No new unit-test is introduced as the existing ones are actually sufficient.

### Why are the changes needed?
RLE float/double data in parquet files will not be read back correctly on big endian platforms.

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

### How was this patch tested?
All unit tests (mvn test) were ran and OK.

Closes #29383 from tinhto-000/SPARK-31703.

Authored-by: Tin Hang To <tinto@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 06:39:10 +00:00
angerszhu 4cf8c1d07d [SPARK-32400][SQL] Improve test coverage of HiveScriptTransformationExec
### What changes were proposed in this pull request?

1. Extract common test case (no serde) to BasicScriptTransformationExecSuite
2. Add more test case for no serde mode about supported data type and behavior in `BasicScriptTransformationExecSuite`
3. Add more test case for hive serde mode about supported type and behavior in `HiveScriptTransformationExecSuite`

### Why are the changes needed?
Improve test coverage of Script Transformation

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

### How was this patch tested?
Added UT

Closes #29401 from AngersZhuuuu/SPARK-32400.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-12 06:02:42 +00:00
Max Gekk 0477d23467 [SPARK-32594][SQL] Fix serialization of dates inserted to Hive tables
### What changes were proposed in this pull request?
Fix `DaysWritable` by overriding parent's method `def get(doesTimeMatter: Boolean): Date` from `DateWritable` instead of `Date get()` because the former one uses the first one. The bug occurs because `HiveOutputWriter.write()` call `def get(doesTimeMatter: Boolean): Date` transitively with default implementation from the parent class  `DateWritable` which doesn't respect date rebases and uses not initialized `daysSinceEpoch` (0 which `1970-01-01`).

### Why are the changes needed?
The changes fix the bug:
```sql
spark-sql> CREATE TABLE table1 (d date);
spark-sql> INSERT INTO table1 VALUES (date '2020-08-11');
spark-sql> SELECT * FROM table1;
1970-01-01
```
The expected result of the last SQL statement must be **2020-08-11** but got **1970-01-01**.

### Does this PR introduce _any_ user-facing change?
Yes. After the fix, `INSERT` work correctly:
```sql
spark-sql> SELECT * FROM table1;
2020-08-11
```

### How was this patch tested?
Add new test to `HiveSerDeReadWriteSuite`

Closes #29409 from MaxGekk/insert-date-into-hive-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 13:32:16 +09:00
Yuming Wang 5d130f0360 [SPARK-32586][SQL] Fix NumberFormatException error message when ansi is enabled
### What changes were proposed in this pull request?

This pr fixes the error message of `NumberFormatException` when casting invalid input to FractionalType and enabling **ansi**:
```
spark-sql> set spark.sql.ansi.enabled=true;
spark.sql.ansi.enabled	true
spark-sql> create table SPARK_32586 using parquet as select 's' s;
spark-sql> select * from SPARK_32586 where s > 1.13D;
java.lang.NumberFormatException: invalid input syntax for type numeric: columnartorow_value_0
```

After this pr:
```
spark-sql> select * from SPARK_32586 where s > 1.13D;
java.lang.NumberFormatException: invalid input syntax for type numeric: s
```

### Why are the changes needed?

Improve error message.

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

No.

### How was this patch tested?

Unit test.

Closes #29405 from wangyum/SPARK-32586.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-12 13:16:57 +09:00
gengjiaan e7c1204f6c [SPARK-32540][SQL] Eliminate the filter clause in aggregate
### What changes were proposed in this pull request?
Spark SQL supported filter clause in aggregate, for example:
`select sum(distinct id) filter (where sex = 'man') from student;`
But sometimes we can eliminate the filter clause in aggregate.
`SELECT COUNT(DISTINCT 1) FILTER (WHERE true) FROM testData;`
could be transformed to
`SELECT COUNT(DISTINCT 1) FROM testData;`
`SELECT COUNT(DISTINCT 1) FILTER (WHERE false) FROM testData;`
could be transformed to
`SELECT 0 FROM testData;`

### Why are the changes needed?
Optimize the filter clause in aggregation

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

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

Closes #29369 from beliefer/eliminate-filter-clause.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 16:20:19 +00:00
xuewei.linxuewei c37357a092 [SPARK-32573][SQL] Anti Join Improvement with EmptyHashedRelation and EmptyHashedRelationWithAllNullKeys
### What changes were proposed in this pull request?
In [SPARK-32290](https://issues.apache.org/jira/browse/SPARK-32290), we introduced several new types of HashedRelation.

* EmptyHashedRelation
* EmptyHashedRelationWithAllNullKeys

They were all limited to used only in NAAJ scenario. These new HashedRelation could be applied to other scenario for performance improvements.

* EmptyHashedRelation could also be used in Normal AntiJoin for fast stop
* While AQE is on and buildSide is EmptyHashedRelationWithAllNullKeys, can convert NAAJ to a Empty LocalRelation to skip meaningless data iteration since in Single-Key NAAJ, if null key exists in BuildSide, will drop all records in streamedSide.

This Patch including two changes.

* using EmptyHashedRelation to do fast stop for common anti join as well
* In AQE, eliminate BroadcastHashJoin(NAAJ) if buildSide is a EmptyHashedRelationWithAllNullKeys

### Why are the changes needed?
LeftAntiJoin could apply `fast stop` when BuildSide is EmptyHashedRelation, While within AQE with EmptyHashedRelationWithAllNullKeys, we can eliminate the NAAJ. This should be a performance improvement in AntiJoin.

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

### How was this patch tested?

* added case in AdaptiveQueryExecSuite.
* added case in HashedRelationSuite.
* Make sure SubquerySuite JoinSuite SQLQueryTestSuite passed.

Closes #29389 from leanken/leanken-SPARK-32573.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 06:23:51 +00:00
allisonwang-db 1b7443bd9a [SPARK-32216][SQL] Remove redundant ProjectExec
### What changes were proposed in this pull request?
This PR added a physical rule to remove redundant project nodes. A `ProjectExec` is redundant when
1. It has the same output attributes and order as its child's output when ordering of these attributes is required.
2. It has the same output attributes as its child's output when attribute output ordering is not required.

For example:
After Filter:
```
== Physical Plan ==
*(1) Project [a#14L, b#15L, c#16, key#17]
+- *(1) Filter (isnotnull(a#14L) AND (a#14L > 5))
   +- *(1) ColumnarToRow
      +- FileScan parquet [a#14L,b#15L,c#16,key#17]
```
The `Project a#14L, b#15L, c#16, key#17` is redundant because its output is exactly the same as filter's output.

Before Aggregate:
```
== Physical Plan ==
*(2) HashAggregate(keys=[key#17], functions=[sum(a#14L), last(b#15L, false)], output=[sum_a#39L, key#17, last_b#41L])
+- Exchange hashpartitioning(key#17, 5), true, [id=#77]
   +- *(1) HashAggregate(keys=[key#17], functions=[partial_sum(a#14L), partial_last(b#15L, false)], output=[key#17, sum#49L, last#50L, valueSet#51])
      +- *(1) Project [key#17, a#14L, b#15L]
         +- *(1) Filter (isnotnull(a#14L) AND (a#14L > 100))
            +- *(1) ColumnarToRow
               +- FileScan parquet [a#14L,b#15L,key#17]
```
The `Project key#17, a#14L, b#15L` is redundant because hash aggregate doesn't require child plan's output to be in a specific order.

### Why are the changes needed?

It removes unnecessary query nodes and makes query plan cleaner.

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

No

### How was this patch tested?

Unit tests

Closes #29031 from allisonwang-db/remove-project.

Lead-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Co-authored-by: allisonwang-db <allison.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-11 03:14:15 +00:00
Wenchen Fan 8659ec554f [SPARK-32469][SQL] ApplyColumnarRulesAndInsertTransitions should be idempotent
### What changes were proposed in this pull request?

This PR makes `ApplyColumnarRulesAndInsertTransitions` idempotent (assuming the custom columnar rules are also idempotent).

### Why are the changes needed?

It's good hygiene to keep rules idempotent

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

No

### How was this patch tested?

new suite

Closes #29273 from cloud-fan/rule.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 16:41:13 +00:00
angerszhu d251443a02 [SPARK-32403][SQL] Refactor current ScriptTransformationExec
# What changes were proposed in this pull request?

This PR comes from the comment: #29085 (comment)

- Extract common Script IOSchema `ScriptTransformationIOSchema`
- avoid repeated judgement extract process output row method `createOutputIteratorWithoutSerde` && `createOutputIteratorWithSerde`
- add default no serde IO schemas `ScriptTransformationIOSchema.defaultIOSchema`

### Why are the changes needed?
Refactor code

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

### How was this patch tested?
NO

Closes #29199 from AngersZhuuuu/spark-32105-followup.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 16:37:31 +00:00
LantaoJin f80a480ee5 [SPARK-32537][SQL][TEST] Add a CTEHintSuite for test coverage
### What changes were proposed in this pull request?
Add a new test suite `CTEHintSuite`

### Why are the changes needed?
This ticket is to address the below comments to help us understand the test coverage of SQL HINT for CTE.
https://github.com/apache/spark/pull/29062#discussion_r463247491
https://github.com/apache/spark/pull/29062#discussion_r463248167

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

### How was this patch tested?
Add a test suite.

Closes #29359 from LantaoJin/SPARK-32537.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 14:04:09 +00:00
Yuanjian Li b03761e330 [SPARK-32456][SS] Check the Distinct by assuming it as Aggregate for Structured Streaming
### What changes were proposed in this pull request?
Check the Distinct nodes by assuming it as Aggregate in `UnsupportOperationChecker` for streaming.

### Why are the changes needed?
We want to fix 2 things here:

1. Give better error message for Distinct related operations in append mode that doesn't have a watermark

We use the union streams as the example, distinct in SQL has the same issue. Since the union clause in SQL has the requirement of deduplication, the parser will generate `Distinct(Union)` and the optimizer rule `ReplaceDistinctWithAggregate` will change it to `Aggregate(Union)`. This logic is of both batch and streaming queries. However, in the streaming, the aggregation will be wrapped by state store operations so we need extra checking logic in `UnsupportOperationChecker`.

Before this change, the SS union queries in Append mode will get the following confusing error when the watermark is lacking.
```
java.util.NoSuchElementException: None.get
	at scala.None$.get(Option.scala:529)
	at scala.None$.get(Option.scala:527)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec.$anonfun$doExecute$9(statefulOperators.scala:346)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.util.Utils$.timeTakenMs(Utils.scala:561)
	at org.apache.spark.sql.execution.streaming.StateStoreWriter.timeTakenMs(statefulOperators.scala:112)
...
```

2. Make `Distinct` in complete mode runnable.

Before this fix, the distinct in complete mode will throw the exception:
```
Complete output mode not supported when there are no streaming aggregations on streaming DataFrames/Datasets;
```

### Does this PR introduce _any_ user-facing change?
Yes, return a better error message.

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

Closes #29256 from xuanyuanking/SPARK-32456.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-10 14:01:31 +09:00
allisonwang-db 924c161544 [SPARK-32337][SQL] Show initial plan in AQE plan tree string
### What changes were proposed in this pull request?
This PR adds initial plan in `AdaptiveSparkPlanExec` and generates tree string for both current plan and initial plan. When the adaptive plan is not final, `Current Plan` will be used to indicate current physical plan, and `Final Plan` will be used when the adaptive plan is final. The difference between `Current Plan` and `Final Plan` here is that current plan indicates an intermediate state. The plan is subject to further transformations, while final plan represents an end state, which means the plan will no longer be changed.

Examples:

Before this change:
```
AdaptiveSparkPlan isFinalPlan=true
+- *(3) BroadcastHashJoin
   :- BroadcastQueryStage 2
       ...
```
`EXPLAIN FORMATTED`
```
== Physical Plan ==
AdaptiveSparkPlan (9)
+- BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
```

After this change
```
AdaptiveSparkPlan isFinalPlan=true
+- == Final Plan ==
   *(3) BroadcastHashJoin
   :- BroadcastQueryStage 2
   :  +- BroadcastExchange
           ...
+- == Initial Plan ==
   SortMergeJoin
   :- Sort
   :  +- Exchange
           ...
```

`EXPLAIN FORMATTED`
```
== Physical Plan ==
AdaptiveSparkPlan (9)
+- == Current Plan ==
   BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
+- == Initial Plan ==
   BroadcastHashJoin Inner BuildRight (8)
   :- Project (3)
   :  +- Filter (2)
```

### Why are the changes needed?
It provides better visibility into the plan change introduced by AQE.

### Does this PR introduce _any_ user-facing change?
Yes. It changed the AQE plan output string.

### How was this patch tested?
Unit test

Closes #29137 from allisonwang-db/aqe-plan.

Lead-authored-by: allisonwang-db <allison.wang@databricks.com>
Co-authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-10 04:49:37 +00:00
Jungtaek Lim (HeartSaVioR) d08e73d0d3 [SPARK-30866][SS] FileStreamSource: Cache fetched list of files beyond maxFilesPerTrigger as unread files
### What changes were proposed in this pull request?

This patch caches the fetched list of files in FileStreamSource to avoid re-fetching whenever possible.

This improvement would be effective when the source options are being set to below:

* `maxFilesPerTrigger` is set
* `latestFirst` is set to `false` (default)

as

* if `maxFilesPerTrigger` is unset, Spark will process all the new files within a batch
* if `latestFirst` is set to `true`, it intends to process "latest" files which Spark has to refresh for every batch

Fetched list of files are filtered against SeenFilesMap before caching - unnecessary files are filtered in this phase. Once we cached the file, we don't check the file again for `isNewFile`, as Spark processes the files in timestamp order so cached files should have equal or later timestamp than latestTimestamp in SeenFilesMap.

Cache is only persisted in memory to simplify the logic - if we support restore cache when restarting query, we should deal with the changes of source options.

To avoid tiny set of inputs on the batch due to have tiny unread files (that could be possible when the list operation provides slightly more than the max files), this patch employs the "lower-bar" to determine whether it's helpful to retain unread files. Spark will discard unread files and do listing in the next batch if the number of unread files is lower than the specific (20% for now) ratio of max files.

This patch will have synergy with SPARK-20568 - while this patch helps to avoid redundant cost of listing, SPARK-20568 will get rid of the cost of listing for processed files. Once the query processes all files in initial load, the cost of listing for the files in initial load will be gone.

### Why are the changes needed?

Spark spends huge cost to fetch the list of files from input paths, but restricts the usage of list in a batch. If the streaming query starts from huge input data for various reasons (initial load, reprocessing, etc.) the cost to fetch the files will be applied to all batches as it is unusual to let first microbatch to process all of initial load.

SPARK-20568 will help to reduce the cost to fetch as processed files will be either deleted or moved outside of input paths, but it still won't help in early phase.

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

Yes, the driver process would require more memory than before if maxFilesPerTrigger is set and latestFirst is set to "false" to cache fetched files. Previously Spark only takes some amount from left side of the list and discards remaining - so technically the peak memory would be same, but they can be freed sooner.

It may not hurt much, as peak memory is still be similar, and it would require similar amount of memory in any way when maxFilesPerTrigger is unset.

### How was this patch tested?

New unit tests. Manually tested under the test environment:

* input files
  * 171,839 files distributed evenly into 24 directories
  * each file contains 200 lines
* query: read from the "file stream source" and repartition to 50, and write to the "file stream sink"
  * maxFilesPerTrigger is set to 100

> before applying the patch

![Screen Shot 2020-02-18 at 11 53 12 PM](https://user-images.githubusercontent.com/1317309/74747932-139a8380-52ab-11ea-8920-26a40070ec32.png)

> after applying the patch

![Screen Shot 2020-02-18 at 11 56 01 PM](https://user-images.githubusercontent.com/1317309/74747860-f5cd1e80-52aa-11ea-89dd-da24e9f6ab00.png)

The area of brown color represents "latestOffset" where listing operation is performed for FileStreamSource. After the patch the cost for listing is paid "only once", whereas before the patch
it was for "every batch".

Closes #27620 from HeartSaVioR/SPARK-30866.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-09 20:14:15 -07:00
kujon 0ae94ad32f [SPARK-32576][SQL] Support PostgreSQL bpchar type and array of char type
### What changes were proposed in this pull request?
This PR fixes the support for char(n)[], character(n)[] data types. Prior to this change, a user would get `Unsupported type ARRAY` exception when attempting to interact with the table with such types.

The description is a bit more detailed in the [JIRA](https://issues.apache.org/jira/browse/SPARK-32393) itself, but the crux of the issue is that postgres driver names char and character types as `bpchar`. The relevant driver code can be found [here](https://github.com/pgjdbc/pgjdbc/blob/master/pgjdbc/src/main/java/org/postgresql/jdbc/TypeInfoCache.java#L85-L87). `char` is very likely to be still needed, as it seems that pg makes a distinction between `char(1)` and `char(n > 1)` as per [this code](b7fd9f3cef/pgjdbc/src/main/java/org/postgresql/core/Oid.java (L64)).

### Why are the changes needed?
For completeness of the pg dialect support.

### Does this PR introduce _any_ user-facing change?
Yes, successful reads of tables with bpchar array instead of errors after this fix.

### How was this patch tested?
Unit tests

Closes #29192 from kujon/fix_postgres_bpchar_array_support.

Authored-by: kujon <jakub.korzeniowski@vortexa.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-09 19:03:37 -07:00
Takeshi Yamamuro 1df855bef2 [SPARK-32564][SQL][TEST][FOLLOWUP] Re-enable TPCDSQuerySuite with empty tables
### What changes were proposed in this pull request?

This is the follow-up PR of #29384 to address the cloud-fan comment: https://github.com/apache/spark/pull/29384#issuecomment-670595111
This PR re-enables `TPCDSQuerySuite` with empty tables for better test coverages.

### Why are the changes needed?

For better test coverage.

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

No.

### How was this patch tested?

Existing tests.

Closes #29391 from maropu/SPARK-32564-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 16:33:25 -07:00
Jungtaek Lim (HeartSaVioR) 8062c1f777 [SPARK-32555][SQL] Add unique ID on query execution
### What changes were proposed in this pull request?

This PR adds unique ID on QueryExecution, so that listeners can leverage the ID to deduplicate redundant calls.

### Why are the changes needed?

I've observed that Spark calls QueryExecutionListener multiple times on same QueryExecution instance (even same funcName for onSuccess). There's no unique ID on QueryExecution, hence it's a bit tricky if the listener would like to deal with same query execution only once.

Note that streaming query has both query ID and run ID which can be leveraged as unique ID.

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

Yes for who uses query execution listener - they'll see `id` field in QueryExecution and leverage it.

### How was this patch tested?

Manually tested. I think the change is obvious hence don't think it warrants a new UT. StreamingQueryListener has been using UUID as `queryId` and `runId` so it should work for the same.

Closes #29372 from HeartSaVioR/SPARK-32555.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-08 15:09:22 -07:00
Takeshi Yamamuro 5b8444af0d [SPARK-32564][SQL][TEST] Inject data statistics to simulate plan generation on actual TPCDS data
### What changes were proposed in this pull request?

`TPCDSQuerySuite` currently computes plans with empty TPCDS tables, then checks if plans can be generated correctly. But, the generated plans can be different from actual ones because the input tables are empty (e.g., the plans always use broadcast-hash joins, but actual ones use sort-merge joins for larger tables). To mitigate the issue, this PR defines data statistics constants extracted from generated TPCDS data in `TPCDSTableStats`, then injects the statistics via `spark.sessionState.catalog.alterTableStats` when defining TPCDS tables in `TPCDSQuerySuite`.

Please see a link below about how to extract the table statistics:
 - https://gist.github.com/maropu/f553d32c323ee803d39e2f7fa0b5a8c3

For example, the generated plans of TPCDS `q2` are different with/without this fix:
```
==== w/ this fix: q2 ====
== Physical Plan ==
* Sort (43)
+- Exchange (42)
   +- * Project (41)
      +- * SortMergeJoin Inner (40)
         :- * Sort (28)
         :  +- Exchange (27)
         :     +- * Project (26)
         :        +- * BroadcastHashJoin Inner BuildRight (25)
         :           :- * HashAggregate (19)
         :           :  +- Exchange (18)
         :           :     +- * HashAggregate (17)
         :           :        +- * Project (16)
         :           :           +- * BroadcastHashJoin Inner BuildRight (15)
         :           :              :- Union (9)
         :           :              :  :- * Project (4)
         :           :              :  :  +- * Filter (3)
         :           :              :  :     +- * ColumnarToRow (2)
         :           :              :  :        +- Scan parquet default.web_sales (1)
         :           :              :  +- * Project (8)
         :           :              :     +- * Filter (7)
         :           :              :        +- * ColumnarToRow (6)
         :           :              :           +- Scan parquet default.catalog_sales (5)
         :           :              +- BroadcastExchange (14)
         :           :                 +- * Project (13)
         :           :                    +- * Filter (12)
         :           :                       +- * ColumnarToRow (11)
         :           :                          +- Scan parquet default.date_dim (10)
         :           +- BroadcastExchange (24)
         :              +- * Project (23)
         :                 +- * Filter (22)
         :                    +- * ColumnarToRow (21)
         :                       +- Scan parquet default.date_dim (20)
         +- * Sort (39)
            +- Exchange (38)
               +- * Project (37)
                  +- * BroadcastHashJoin Inner BuildRight (36)
                     :- * HashAggregate (30)
                     :  +- ReusedExchange (29)
                     +- BroadcastExchange (35)
                        +- * Project (34)
                           +- * Filter (33)
                              +- * ColumnarToRow (32)
                                 +- Scan parquet default.date_dim (31)

==== w/o this fix: q2 ====
== Physical Plan ==
* Sort (40)
+- Exchange (39)
   +- * Project (38)
      +- * BroadcastHashJoin Inner BuildRight (37)
         :- * Project (26)
         :  +- * BroadcastHashJoin Inner BuildRight (25)
         :     :- * HashAggregate (19)
         :     :  +- Exchange (18)
         :     :     +- * HashAggregate (17)
         :     :        +- * Project (16)
         :     :           +- * BroadcastHashJoin Inner BuildRight (15)
         :     :              :- Union (9)
         :     :              :  :- * Project (4)
         :     :              :  :  +- * Filter (3)
         :     :              :  :     +- * ColumnarToRow (2)
         :     :              :  :        +- Scan parquet default.web_sales (1)
         :     :              :  +- * Project (8)
         :     :              :     +- * Filter (7)
         :     :              :        +- * ColumnarToRow (6)
         :     :              :           +- Scan parquet default.catalog_sales (5)
         :     :              +- BroadcastExchange (14)
         :     :                 +- * Project (13)
         :     :                    +- * Filter (12)
         :     :                       +- * ColumnarToRow (11)
         :     :                          +- Scan parquet default.date_dim (10)
         :     +- BroadcastExchange (24)
         :        +- * Project (23)
         :           +- * Filter (22)
         :              +- * ColumnarToRow (21)
         :                 +- Scan parquet default.date_dim (20)
         +- BroadcastExchange (36)
            +- * Project (35)
               +- * BroadcastHashJoin Inner BuildRight (34)
                  :- * HashAggregate (28)
                  :  +- ReusedExchange (27)
                  +- BroadcastExchange (33)
                     +- * Project (32)
                        +- * Filter (31)
                           +- * ColumnarToRow (30)
                              +- Scan parquet default.date_dim (29)
```

This comes from the cloud-fan comment: https://github.com/apache/spark/pull/29270#issuecomment-666098964

### Why are the changes needed?

For better test coverage.

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

No.

### How was this patch tested?

Existing tests.

Closes #29384 from maropu/AddTPCDSTableStats.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-07 08:17:45 -07:00
Liang-Chi Hsieh 7b6e1d5cec [SPARK-25557][SQL] Nested column predicate pushdown for ORC
### What changes were proposed in this pull request?

We added nested column predicate pushdown for Parquet in #27728. This patch extends the feature support to ORC.

### Why are the changes needed?

Extending the feature to ORC for feature parity. Better performance for handling nested predicate pushdown.

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

No

### How was this patch tested?

Unit tests.

Closes #28761 from viirya/SPARK-25557.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-08-07 08:07:41 -07:00
Wenchen Fan d5682c13a2 [SPARK-32528][SQL][TEST] The analyze method should make sure the plan is analyzed
### What changes were proposed in this pull request?

This PR updates the `analyze` method to make sure the plan can be resolved. It also fixes some miswritten optimizer tests.

### Why are the changes needed?

It's error-prone if the `analyze` method can return an unresolved plan.

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

no

### How was this patch tested?

test only

Closes #29349 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-07 07:36:08 +00:00
GuoPhilipse aa4d3c19fe [SPARK-32560][SQL] Improve exception message at InsertIntoHiveTable.processInsert
### What changes were proposed in this pull request?
improve exception message

### Why are the changes needed?
the before message lack of single quotes, we may improve it to keep consisent.
![image](https://user-images.githubusercontent.com/46367746/89595808-15bbc300-d888-11ea-9914-b05ea7b66461.png)

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

### How was this patch tested?
No ,it is only improving the message.

Closes #29376 from GuoPhilipse/improve-exception-message.

Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-07 14:29:32 +09:00
wangguangxin.cn 9a35b93c8a [SPARK-32559][SQL] Fix the trim logic in UTF8String.toInt/toLong did't handle non-ASCII characters correctly
### What changes were proposed in this pull request?
The trim logic in Cast expression introduced in https://github.com/apache/spark/pull/26622 trim non-ASCII characters unexpectly.

Before this patch
![image](https://user-images.githubusercontent.com/1312321/89513154-caad9b80-d806-11ea-9ebe-17c9e7d1b5b3.png)

After this patch
![image](https://user-images.githubusercontent.com/1312321/89513196-d731f400-d806-11ea-959c-6a7dc29dcd49.png)

### Why are the changes needed?
The behavior described above doesn't make sense, and also doesn't consistent with the behavior when cast a string to double/float, as well as doesn't consistent with the behavior of Hive

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

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

Closes #29375 from WangGuangxin/cast-bugfix.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-07 05:00:33 +00:00
Max Gekk 6664e282f6 [SPARK-32546][SQL][FOLLOWUP] Add .toSeq to tableNames in HiveClientImpl.listTablesByType
### What changes were proposed in this pull request?
Explicitly convert `tableNames` to `Seq` in `HiveClientImpl.listTablesByType` as it was done by c28a6fa511 (diff-6fd847124f8eae45ba2de1cf7d6296feR769)

### Why are the changes needed?
See this PR https://github.com/apache/spark/pull/29111, to compile by Scala 2.13. The changes were discarded by https://github.com/apache/spark/pull/29363 accidentally.

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

### How was this patch tested?
Compiling by Scala 2.13

Closes #29379 from MaxGekk/fix-listTablesByType-for-views-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 14:23:56 +00:00
Max Gekk dc96f2f8d6 [SPARK-32546][SQL] Get table names directly from Hive tables
### What changes were proposed in this pull request?
Get table names directly from a sequence of Hive tables in `HiveClientImpl.listTablesByType()` by skipping conversions Hive tables to Catalog tables.

### Why are the changes needed?
A Hive metastore can be shared across many clients. A client can create tables using a SerDe which is not available on other clients, for instance `ROW FORMAT SERDE "com.ibm.spss.hive.serde2.xml.XmlSerDe"`. In the current implementation, other clients get the following exception while getting views:
```
java.lang.RuntimeException: MetaException(message:java.lang.ClassNotFoundException Class com.ibm.spss.hive.serde2.xml.XmlSerDe not found)
```
when `com.ibm.spss.hive.serde2.xml.XmlSerDe` is not available.

### Does this PR introduce _any_ user-facing change?
Yes. For example, `SHOW VIEWS` returns a list of views instead of throwing an exception.

### How was this patch tested?
- By existing test suites like:
```
$ build/sbt -Phive-2.3 "test:testOnly org.apache.spark.sql.hive.client.VersionsSuite"
```
- And manually:

1. Build Spark with Hive 1.2: `./build/sbt package -Phive-1.2 -Phive -Dhadoop.version=2.8.5`

2. Run spark-shell with a custom Hive SerDe, for instance download [json-serde-1.3.8-jar-with-dependencies.jar](https://github.com/cdamak/Twitter-Hive/blob/master/json-serde-1.3.8-jar-with-dependencies.jar) from https://github.com/cdamak/Twitter-Hive:
```
$ ./bin/spark-shell --jars ../Downloads/json-serde-1.3.8-jar-with-dependencies.jar
```

3. Create a Hive table using this SerDe:
```scala
scala> :paste
// Entering paste mode (ctrl-D to finish)

sql(s"""
  |CREATE TABLE json_table2(page_id INT NOT NULL)
  |ROW FORMAT SERDE 'org.openx.data.jsonserde.JsonSerDe'
  |""".stripMargin)

// Exiting paste mode, now interpreting.
res0: org.apache.spark.sql.DataFrame = []

scala> sql("SHOW TABLES").show
+--------+-----------+-----------+
|database|  tableName|isTemporary|
+--------+-----------+-----------+
| default|json_table2|      false|
+--------+-----------+-----------+

scala> sql("SHOW VIEWS").show
+---------+--------+-----------+
|namespace|viewName|isTemporary|
+---------+--------+-----------+
+---------+--------+-----------+
```

4. Quit from the current `spark-shell` and run it without jars:
```
$ ./bin/spark-shell
```

5. Show views. Without the fix, it throws the exception:
```scala
scala> sql("SHOW VIEWS").show
20/08/06 10:53:36 ERROR log: error in initSerDe: java.lang.ClassNotFoundException Class org.openx.data.jsonserde.JsonSerDe not found
java.lang.ClassNotFoundException: Class org.openx.data.jsonserde.JsonSerDe not found
	at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2273)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:385)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:276)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializer(Table.java:258)
	at org.apache.hadoop.hive.ql.metadata.Table.getCols(Table.java:605)
```

After the fix:
```scala
scala> sql("SHOW VIEWS").show
+---------+--------+-----------+
|namespace|viewName|isTemporary|
+---------+--------+-----------+
+---------+--------+-----------+
```

Closes #29363 from MaxGekk/fix-listTablesByType-for-views.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 08:35:59 +00:00
Gengliang Wang e93b8f02cd [SPARK-32539][INFRA] Disallow FileSystem.get(Configuration conf) in style check by default
### What changes were proposed in this pull request?

Disallow `FileSystem.get(Configuration conf)` in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem()` instead.

### Why are the changes needed?

The method `FileSystem.get(Configuration conf)` will return a default FileSystem instance if the conf `fs.file.impl` is not set. This can cause file not found exception on reading a target path of non-default file system, e.g. S3. It is hard to discover such a mistake via unit tests.
If we disallow it in Scala style check by default and suggest developers use `FileSystem.get(URI uri, Configuration conf)` or `Path.getFileSystem(Configuration conf)`, we can reduce potential regression and PR review effort.

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

No

### How was this patch tested?

Manually run scala style check and test.

Closes #29357 from gengliangwang/newStyleRule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-06 05:56:59 +00:00
Michael Munday 4a0427cbc1 [SPARK-32485][SQL][TEST] Fix endianness issues in tests in RecordBinaryComparatorSuite
### What changes were proposed in this pull request?
PR #26548 means that RecordBinaryComparator now uses big endian
byte order for long comparisons. However, this means that some of
the constants in the regression tests no longer map to the same
values in the comparison that they used to.

For example, one of the tests does a comparison between
Long.MIN_VALUE and 1 in order to trigger an overflow condition that
existed in the past (i.e. Long.MIN_VALUE - 1). These constants
correspond to the values 0x80..00 and 0x00..01. However on a
little-endian machine the bytes in these values are now swapped
before they are compared. This means that we will now be comparing
0x00..80 with 0x01..00. 0x00..80 - 0x01..00 does not overflow
therefore missing the original purpose of the test.

To fix this the constants are now explicitly written out in big
endian byte order to match the byte order used in the comparison.
This also fixes the tests on big endian machines (which would
otherwise get a different comparison result to the little-endian
machines).

### Why are the changes needed?
The regression tests no longer serve their initial purposes and also fail on big-endian systems.

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

### How was this patch tested?
Tests run on big-endian system (s390x).

Closes #29259 from mundaym/fix-endian.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 16:11:09 +00:00
Kent Yao 1b6f482adb [SPARK-32492][SQL][FOLLOWUP][TEST-MAVEN] Fix jenkins maven jobs
### What changes were proposed in this pull request?

The newly added test fails Jenkins maven jobs, see https://github.com/apache/spark/pull/29303#discussion_r464729021

We move the test from `ThriftServerWithSparkContextSuite` to `SparkMetadataOperationSuite`, the former uses an embedded thrift server where the server and the client are in the same JVM process and the latter forks a new process to start the server where the server and client are isolated.
The sbt runner seems to be fine with the test in the `ThriftServerWithSparkContextSuite`, but the maven runner with `scalates`t plugin will face the classloader issue as we will switch classloader to the one in the `sharedState` which is not the one that hive uses to load some classes. This is more like an issue that belongs to the maven runner or the `scalatest`.
So in this PR, we simply move it to bypass the issue.

BTW, we should test against the way of using embedded thrift server to verify whether it is just a maven issue or not, there could be some use cases with this API.

### Why are the changes needed?

Jenkins recovery

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

no

### How was this patch tested?

modified uts

Closes #29347 from yaooqinn/SPARK-32492-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 13:56:11 +00:00
Max Gekk 3a437ed22b [SPARK-32501][SQL] Convert null to "null" in structs, maps and arrays while casting to strings
### What changes were proposed in this pull request?
Convert `NULL` elements of maps, structs and arrays to the `"null"` string while converting maps/struct/array values to strings. The SQL config `spark.sql.legacy.omitNestedNullInCast.enabled` controls the behaviour. When it is `true`, `NULL` elements of structs/maps/arrays will be omitted otherwise, when it is `false`, `NULL` elements will be converted to `"null"`.

### Why are the changes needed?
1. It is impossible to distinguish empty string and null, for instance:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```
2. Inconsistent NULL conversions for top-level values and nested columns, for instance:
```scala
scala> sql("select named_struct('c', null), null").show
+---------------------+----+
|named_struct(c, NULL)|NULL|
+---------------------+----+
|                   []|null|
+---------------------+----+
```
3. `.show()` is different from conversions to Hive strings, and as a consequence its output is different from `spark-sql` (sql tests):
```sql
spark-sql> select named_struct('c', null) as struct;
{"c":null}
```
```scala
scala> sql("select named_struct('c', null) as struct").show
+------+
|struct|
+------+
|    []|
+------+
```

4. It is impossible to distinguish empty struct/array from struct/array with null in the current implementation:
```scala
scala> Seq[Seq[String]](Seq(), Seq(null)).toDF.show()
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```

### Does this PR introduce _any_ user-facing change?
Yes, before:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+-----+
|value|
+-----+
|   []|
|   []|
+-----+
```

After:
```scala
scala> Seq(Seq(""), Seq(null)).toDF().show
+------+
| value|
+------+
|    []|
|[null]|
+------+
```

### How was this patch tested?
By existing test suite `CastSuite`.

Closes #29311 from MaxGekk/nested-null-to-string.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 12:03:36 +00:00
Huaxin Gao b14a1e2816 [SPARK-32402][SQL] Implement ALTER TABLE in JDBC Table Catalog
### What changes were proposed in this pull request?
Implement ALTER TABLE in JDBC Table Catalog
The following ALTER TABLE are implemented:
```
ALTER TABLE table_name ADD COLUMNS ( column_name datatype [ , ... ] );
ALTER TABLE table_name RENAME COLUMN old_column_name TO new_column_name;
ALTER TABLE table_name DROP COLUMN column_name;
ALTER TABLE table_name ALTER COLUMN column_name TYPE new_type;
ALTER TABLE table_name ALTER COLUMN column_name SET NOT NULL;
```
I haven't checked ALTER TABLE syntax for all the databases yet. I will check. If there are different syntax, I will have a follow-up to override the dialect.

Seems most of the databases don't support updating comments and column position, so I didn't implement UpdateColumnComment and UpdateColumnPosition.

### Why are the changes needed?
Complete the JDBCTableCatalog implementation

### Does this PR introduce _any_ user-facing change?
Yes
`JDBCTableCatalog.alterTable`

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

Closes #29324 from huaxingao/alter_table.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-05 11:57:33 +00:00
Max Gekk 7eb6f45688 [SPARK-32499][SQL] Use {} in conversions maps and structs to strings
### What changes were proposed in this pull request?
Change casting of map and struct values to strings by using the `{}` brackets instead of `[]`. The behavior is controlled by the SQL config `spark.sql.legacy.castComplexTypesToString.enabled`. When it is `true`, `CAST` wraps maps and structs by `[]` in casting to strings. Otherwise, if this is `false`, which is the default, maps and structs are wrapped by `{}`.

### Why are the changes needed?
- To distinguish structs/maps from arrays.
- To make `show`'s output consistent with Hive and conversions to Hive strings.
- To display dataframe content in the same form by `spark-sql` and `show`
- To be consistent with the `*.sql` tests

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

### How was this patch tested?
By existing test suite `CastSuite`.

Closes #29308 from MaxGekk/show-struct-map.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 14:57:09 +00:00
fqaiser94@gmail.com 6d69068057 [SPARK-32521][SQL] Bug-fix: WithFields Expression should not be foldable
### What changes were proposed in this pull request?

Make WithFields Expression not foldable.

### Why are the changes needed?

The following query currently fails on master brach:
```
sql("SELECT named_struct('a', 1, 'b', 2) a")
.select($"a".withField("c", lit(3)).as("a"))
.show(false)
// java.lang.UnsupportedOperationException: Cannot evaluate expression: with_fields(named_struct(a, 1, b, 2), c, 3)
```
This happens because the Catalyst optimizer sees that the WithFields Expression is foldable and tries to statically evaluate the WithFields Expression (via the ConstantFolding rule), however it cannot do so because WithFields Expression is Unevaluable.

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

Yes, queries like the one shared above will now succeed.
That said, this bug was introduced in Spark 3.1.0 which has yet to be released.

### How was this patch tested?

A new unit test was added.

Closes #29338 from fqaiser94/SPARK-32521.

Lead-authored-by: fqaiser94@gmail.com <fqaiser94@gmail.com>
Co-authored-by: fqaiser94 <fqaiser94@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 12:11:04 +00:00
Dongjoon Hyun 7fec6e0c16 [SPARK-32524][SQL][TESTS] CachedBatchSerializerSuite should clean up InMemoryRelation.ser
### What changes were proposed in this pull request?

This PR aims to clean up `InMemoryRelation.ser` in `CachedBatchSerializerSuite`.

### Why are the changes needed?

SPARK-32274 makes SQL cache serialization pluggable.
```
[SPARK-32274][SQL] Make SQL cache serialization pluggable
```

This causes UT failures.
```
$ build/sbt "sql/testOnly *.CachedBatchSerializerSuite *.CachedTableSuite"
...
[info]   Cause: java.lang.IllegalStateException: This does not work. This is only for testing
[info]   at org.apache.spark.sql.execution.columnar.TestSingleIntColumnarCachedBatchSerializer.convertInternalRowToCachedBatch(CachedBatchSerializerSuite.scala:49)
...
[info] *** 30 TESTS FAILED ***
[error] Failed: Total 51, Failed 30, Errors 0, Passed 21
[error] Failed tests:
[error] 	org.apache.spark.sql.CachedTableSuite
[error] (sql/test:testOnly) sbt.TestsFailedException: Tests unsuccessful
```

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

No.

### How was this patch tested?

Manually.
```
$ build/sbt "sql/testOnly *.CachedBatchSerializerSuite *.CachedTableSuite"
[info] Tests: succeeded 51, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[info] Passed: Total 51, Failed 0, Errors 0, Passed 51
```

Closes #29346 from dongjoon-hyun/SPARK-32524-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-04 17:49:52 +09:00
gengjiaan 1597d8fcd4 [SPARK-30276][SQL] Support Filter expression allows simultaneous use of DISTINCT
### What changes were proposed in this pull request?
This PR is related to https://github.com/apache/spark/pull/26656.
https://github.com/apache/spark/pull/26656 only support use FILTER clause on aggregate expression without DISTINCT.
This PR will enhance this feature when one or more DISTINCT aggregate expressions which allows the use of the FILTER clause.
Such as:
```
select sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id) filter (where sex = 'man') from student group by class_id;
select count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student;
select class_id, count(id) filter (where class_id = 1), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select sum(distinct id), sum(distinct id) filter (where sex = 'man') from student;
select class_id, sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
select class_id, count(id), count(id) filter (where class_id = 1), sum(distinct id), sum(distinct id) filter (where sex = 'man') from student group by class_id;
```

### Why are the changes needed?
Spark SQL only support use FILTER clause on aggregate expression without DISTINCT.
This PR support Filter expression allows simultaneous use of DISTINCT

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

### How was this patch tested?
Exists and new UT

Closes #29291 from beliefer/support-distinct-with-filter.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-04 04:41:19 +00:00
Takuya UESHIN 7deb67c28f [SPARK-32160][CORE][PYSPARK][FOLLOWUP] Change the config name to switch allow/disallow SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #29278.
This PR changes the config name to switch allow/disallow `SparkContext` in executors as per the comment https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.

### Why are the changes needed?

The config name `spark.executor.allowSparkContext` is more reasonable.

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

Yes, the config name is changed.

### How was this patch tested?

Updated tests.

Closes #29340 from ueshin/issues/SPARK-32160/change_config_name.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-08-04 12:45:06 +09:00
Max Gekk f3b10f526b [SPARK-32290][SQL][FOLLOWUP] Add version for the SQL config spark.sql.optimizeNullAwareAntiJoin
### What changes were proposed in this pull request?
Add the version `3.1.0` for the SQL config `spark.sql.optimizeNullAwareAntiJoin`.

### Why are the changes needed?
To inform users when the config was added, for example on the page http://spark.apache.org/docs/latest/configuration.html.

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

### How was this patch tested?
By compiling and running `./dev/scalastyle`.

Closes #29335 from MaxGekk/leanken-SPARK-32290-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 16:05:54 +00:00
Takeshi Yamamuro c6109ba918 [SPARK-32257][SQL] Reports explicit errors for invalid usage of SET/RESET command
### What changes were proposed in this pull request?

This PR modified the parser code to handle invalid usages of a SET/RESET command.
For example;
```
SET spark.sql.ansi.enabled true
```
The above SQL command does not change the configuration value and it just tries to display the value of the configuration
`spark.sql.ansi.enabled true`. This PR disallows using special characters including spaces in the configuration name and reports a user-friendly error instead. In the error message, it tells users a workaround to use quotes or a string literal if they still needs to specify a configuration with them. 

Before this PR:
```
scala> sql("SET spark.sql.ansi.enabled true").show(1, -1)
+---------------------------+-----------+
|key                        |value      |
+---------------------------+-----------+
|spark.sql.ansi.enabled true|<undefined>|
+---------------------------+-----------+
```

After this PR:
```
scala> sql("SET spark.sql.ansi.enabled true")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
SET spark.sql.ansi.enabled true
^^^
```

### Why are the changes needed?

For better user-friendly errors.

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

No.

### How was this patch tested?

Added tests in `SparkSqlParserSuite`.

Closes #29146 from maropu/SPARK-32257.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 13:00:07 +00:00
Kent Yao 7f5326c082 [SPARK-32492][SQL] Fulfill missing column meta information COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION for thriftserver client tools
### What changes were proposed in this pull request?

This PR fulfills some missing fields for SparkGetColumnsOperation including COLUMN_SIZE /DECIMAL_DIGITS/NUM_PREC_RADIX/ORDINAL_POSITION

and improve the test coverage.

### Why are the changes needed?

make jdbc tools happier

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

yes,

#### before
![image](https://user-images.githubusercontent.com/8326978/88911764-e78b2180-d290-11ea-8abb-96f137f9c3c4.png)

#### after

![image](https://user-images.githubusercontent.com/8326978/88911709-d04c3400-d290-11ea-90ab-02bda3e628e9.png)

![image](https://user-images.githubusercontent.com/8326978/88912007-39cc4280-d291-11ea-96d6-1ef3abbbddec.png)

### How was this patch tested?

add unit tests

Closes #29303 from yaooqinn/SPARK-32492.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 12:49:34 +00:00
Kent Yao 3deb59d5c2 [SPARK-31709][SQL] Proper base path for database/table location when it is a relative path
### What changes were proposed in this pull request?

Currently, the user home directory is used as the base path for the database and table locations when their locationa are specified with a relative paths, e.g.
```sql
> set spark.sql.warehouse.dir;
spark.sql.warehouse.dir	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/spark-warehouse/
spark-sql> create database loctest location 'loctestdbdir';

spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Owner	kentyao

spark-sql> create table loctest(id int) location 'loctestdbdir';
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	default
Table	loctest
Owner	kentyao
Created Time	Thu May 14 16:29:05 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200512/loctestdbdir
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
The user home is not always warehouse-related, unchangeable in runtime, and shared both by database and table as the parent directory. Meanwhile, we use the table path as the parent directory for relative partition locations.

The config `spark.sql.warehouse.dir` represents `the default location for managed databases and tables`.
For databases, the case above seems not to follow its semantics, because it should use ` `spark.sql.warehouse.dir` as the base path instead.

For tables, it seems to be right but here I suggest enriching the meaning that lets it also be the for external tables with relative paths for locations.

With changes in this PR,

The location of a database will be `warehouseDir/dbpath` when `dbpath` is relative.
The location of a table will be `dbpath/tblpath` when `tblpath` is relative.

### Why are the changes needed?

bugfix and improvement

Firstly, the databases with relative locations should be created under the default location specified by `spark.sql.warehouse.dir`.

Secondly, the external tables with relative paths may also follow this behavior for consistency.

At last, the behavior for database, tables and partitions with relative paths to choose base paths should be the same.

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

Yes, this PR changes the `createDatabase`, `alterDatabase`, `createTable` and `alterTable` APIs and related DDLs. If the LOCATION clause is followed by a relative path, the root path will be `spark.sql.warehouse.dir` for databases, and `spark.sql.warehouse.dir` / `dbPath` for tables.

e.g.

#### after
```sql
spark-sql> desc database loctest;
Database Name	loctest
Comment
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest
Owner	kentyao
spark-sql> use loctest;
spark-sql> create table loctest(id int) location 'loctest';
20/05/14 18:18:02 WARN InMemoryFileIndex: The directory file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/loctest was not found. Was it deleted very recently?
20/05/14 18:18:02 WARN SessionState: METASTORE_FILTER_HOOK will be ignored, since hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.internal.ss.authz.settings.applied.marker does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/05/14 18:18:03 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
spark-sql> alter table loctest set location 'loctest2'
         > ;
spark-sql> desc formatted loctest;
id	int	NULL

# Detailed Table Information
Database	loctest
Table	loctest
Owner	kentyao
Created Time	Thu May 14 18:18:03 CST 2020
Last Access	UNKNOWN
Created By	Spark 3.1.0-SNAPSHOT
Type	EXTERNAL
Provider	parquet
Location	file:/Users/kentyao/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-SPARK-31709/spark-warehouse/loctest/loctest2
Serde Library	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
```
### How was this patch tested?

Add unit tests.

Closes #28527 from yaooqinn/SPARK-31709.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 12:48:22 +00:00
beliefer 42f9ee4c7d [SPARK-24884][SQL] Support regexp function regexp_extract_all
### What changes were proposed in this pull request?
`regexp_extract_all` is a very useful function expanded the capabilities of `regexp_extract`.
There are some description of this function.
```
SELECT regexp_extract('1a 2b 14m', '\d+', 0); -- 1
SELECT regexp_extract_all('1a 2b 14m', '\d+', 0); -- [1, 2, 14]
SELECT regexp_extract('1a 2b 14m', '(\d+)([a-z]+)', 2); -- 'a'
SELECT regexp_extract_all('1a 2b 14m', '(\d+)([a-z]+)', 2); -- ['a', 'b', 'm']
```
There are some mainstream database support the syntax.
**Presto:**
https://prestodb.io/docs/current/functions/regexp.html

**Pig:**
https://pig.apache.org/docs/latest/api/org/apache/pig/builtin/REGEX_EXTRACT_ALL.html

Note: This PR pick up the work of https://github.com/apache/spark/pull/21985
### Why are the changes needed?
`regexp_extract_all` is a very useful function and make work easier.

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

### How was this patch tested?
New UT

Closes #27507 from beliefer/support-regexp_extract_all.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 06:03:55 +00:00
Prakhar Jain 7a09e71198 [SPARK-32509][SQL] Ignore unused DPP True Filter in Canonicalization
### What changes were proposed in this pull request?
This PR fixes issues relate to Canonicalization of FileSourceScanExec when it contains unused DPP Filter.

### Why are the changes needed?

As part of PlanDynamicPruningFilter rule, the unused DPP Filter are simply replaced by `DynamicPruningExpression(TrueLiteral)` so that they can be avoided. But these unnecessary`DynamicPruningExpression(TrueLiteral)` partition filter inside the FileSourceScanExec affects the canonicalization of the node and so in many cases, this can prevent ReuseExchange from happening.

This PR fixes this issue by ignoring the unused DPP filter in the `def doCanonicalize` method.

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

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

Closes #29318 from prakharjain09/SPARK-32509_df_reuse.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:26:03 +00:00
Max Gekk fda397d9c8 [SPARK-32510][SQL] Check duplicate nested columns in read from JDBC datasource
### What changes were proposed in this pull request?
Check that there are not duplicate column names on the same level (top level or nested levels) in reading from JDBC datasource. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value:
```
The check takes into account the SQL config `spark.sql.caseSensitive` (`false` by default).

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758, and duplicates in nested structures by https://github.com/apache/spark/pull/29234.

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

### How was this patch tested?
Added new test suite `JdbcNestedDataSourceSuite`.

Closes #29317 from MaxGekk/jdbc-dup-nested-columns.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:20:17 +00:00
Robert (Bobby) Evans 713124d5e3 [SPARK-32274][SQL] Make SQL cache serialization pluggable
### What changes were proposed in this pull request?

Add a config to let users change how SQL/Dataframe data is compressed when cached.

This adds a few new classes/APIs for use with this config.

1. `CachedBatch` is a trait used to tag data that is intended to be cached. It has a few APIs that lets us keep the compression/serialization of the data separate from the metrics about it.
2. `CachedBatchSerializer` provides the APIs that must be implemented to cache data.
    * `convertForCache` is an API that runs a cached spark plan and turns its result into an `RDD[CachedBatch]`.  The actual caching is done outside of this API
   * `buildFilter` is an API that takes a set of predicates and builds a filter function that can be used to filter the `RDD[CachedBatch]` returned by `convertForCache`
   * `decompressColumnar` decompresses an `RDD[CachedBatch]` into an `RDD[ColumnarBatch]` This is only used for a limited set of data types.  These data types may expand in the future.  If they do we can add in a new API with a default value that says which data types this serializer supports.
   * `decompressToRows` decompresses an `RDD[CachedBatch]` into an `RDD[InternalRow]` this API, like `decompressColumnar` decompresses the data in `CachedBatch` but turns it into `InternalRow`s, typically using code generation for performance reasons.

There is also an API that lets you reuse the current filtering based on min/max values. `SimpleMetricsCachedBatch` and `SimpleMetricsCachedBatchSerializer`.

### Why are the changes needed?

This lets users explore different types of compression and compression ratios.

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

This adds in a single config, and exposes some developer API classes described above.

### How was this patch tested?

I ran the unit tests around this and I also did some manual performance tests.  I could find any performance difference between the old and new code, and if there is any it is within error.

Closes #29067 from revans2/pluggable_cache_serializer.

Authored-by: Robert (Bobby) Evans <bobby@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-08-03 03:15:54 +00:00
Wenchen Fan 1c6dff7b5f [SPARK-32083][SQL] AQE coalesce should at least return one partition
### What changes were proposed in this pull request?

This PR updates the AQE framework to at least return one partition during coalescing.

This PR also updates `ShuffleExchangeExec.canChangeNumPartitions` to not coalesce for `SinglePartition`.

### Why are the changes needed?

It's a bit risky to return 0 partitions, as sometimes it's different from empty data. For example, global aggregate will return one result row even if the input table is empty. If there is 0 partition, no task will be run and no result will be returned. More specifically, the global aggregate requires `AllTuples` and we can't coalesce to 0 partitions.

This is not a real bug for now. The global aggregate will be planned as partial and final physical agg nodes. The partial agg will return at least one row, so that the shuffle still have data. But it's better to fix this issue to avoid potential bugs in the future.

According to https://github.com/apache/spark/pull/28916, this change also fix some perf problems.

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

no

### How was this patch tested?

updated test.

Closes #29307 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 14:20:20 +00:00
Yuanjian Li 354313b6bc [SPARK-31894][SS][FOLLOW-UP] Rephrase the config doc
### What changes were proposed in this pull request?
Address comment in https://github.com/apache/spark/pull/28707#discussion_r461102749

### Why are the changes needed?
Hide the implementation details in the config doc.

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

### How was this patch tested?
Document only.

Closes #29315 from xuanyuanking/SPARK-31894-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 14:16:41 +00:00
Kent Yao f4800406a4 [SPARK-32406][SQL][FOLLOWUP] Make RESET fail against static and core configs
### What changes were proposed in this pull request?

This followup addresses comments from https://github.com/apache/spark/pull/29202#discussion_r462054784

1. make RESET static SQL configs/spark core configs fail as same as the SET command. Not that, for core ones, they have to be pre-registered, otherwise, they are still able to be SET/RESET

2. add test cases for configurations w/ optional default values

### Why are the changes needed?

behavior change with suggestions from PMCs

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

Yes, RESET will fail after this PR, before it just does nothing because the static ones are static.

### How was this patch tested?

add more tests.

Closes #29297 from yaooqinn/SPARK-32406-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 22:11:05 +09:00
Takuya UESHIN 8014b0b5d6 [SPARK-32160][CORE][PYSPARK] Add a config to switch allow/disallow to create SparkContext in executors
### What changes were proposed in this pull request?

This is a follow-up of #28986.
This PR adds a config to switch allow/disallow to create `SparkContext` in executors.

- `spark.driver.allowSparkContextInExecutors`

### Why are the changes needed?

Some users or libraries actually create `SparkContext` in executors.
We shouldn't break their workloads.

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

Yes, users will be able to create `SparkContext` in executors with the config enabled.

### How was this patch tested?

More tests are added.

Closes #29278 from ueshin/issues/SPARK-32160/add_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-31 17:28:35 +09:00
Cheng Su ae82768c13 [SPARK-32421][SQL] Add code-gen for shuffled hash join
### What changes were proposed in this pull request?

Adding codegen for shuffled hash join. Shuffled hash join codegen is very similar to broadcast hash join codegen. So most of code change is to refactor existing codegen in `BroadcastHashJoinExec` to `HashJoin`.

Example codegen for query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153):

```
  def shuffleHashJoin(): Unit = {
    val N: Long = 4 << 20
    withSQLConf(
      SQLConf.SHUFFLE_PARTITIONS.key -> "2",
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000",
      SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
      codegenBenchmark("shuffle hash join", N) {
        val df1 = spark.range(N).selectExpr(s"id as k1")
        val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2")
        val df = df1.join(df2, col("k1") === col("k2"))
        assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined)
        df.noop()
      }
    }
  }
```

Shuffled hash join codegen:

```
== Subtree 3 / 3 (maxMethodCodeSize:113; maxConstantPoolSize:126(0.19% used); numInnerClasses:0) ==
*(3) ShuffledHashJoin [k1#2L], [k2#6L], Inner, BuildRight
:- *(1) Project [id#0L AS k1#2L]
:  +- *(1) Range (0, 4194304, step=1, splits=1)
+- *(2) Project [(id#4L * 3) AS k2#6L]
   +- *(2) Range (0, 1398101, step=1, splits=1)

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage3(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=3
/* 006 */ final class GeneratedIteratorForCodegenStage3 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private scala.collection.Iterator inputadapter_input_0;
/* 010 */   private org.apache.spark.sql.execution.joins.HashedRelation shj_relation_0;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] shj_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[1];
/* 012 */
/* 013 */   public GeneratedIteratorForCodegenStage3(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input_0 = inputs[0];
/* 021 */     shj_relation_0 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[0] /* plan */).buildHashedRelation(inputs[1]);
/* 022 */     shj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 023 */
/* 024 */   }
/* 025 */
/* 026 */   private void shj_doConsume_0(InternalRow inputadapter_row_0, long shj_expr_0_0) throws java.io.IOException {
/* 027 */     // generate join key for stream side
/* 028 */
/* 029 */     // find matches from HashRelation
/* 030 */     scala.collection.Iterator shj_matches_0 = false ?
/* 031 */     null : (scala.collection.Iterator)shj_relation_0.get(shj_expr_0_0);
/* 032 */     if (shj_matches_0 != null) {
/* 033 */       while (shj_matches_0.hasNext()) {
/* 034 */         UnsafeRow shj_matched_0 = (UnsafeRow) shj_matches_0.next();
/* 035 */         {
/* 036 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[1] /* numOutputRows */).add(1);
/* 037 */
/* 038 */           long shj_value_1 = shj_matched_0.getLong(0);
/* 039 */           shj_mutableStateArray_0[0].reset();
/* 040 */
/* 041 */           shj_mutableStateArray_0[0].write(0, shj_expr_0_0);
/* 042 */
/* 043 */           shj_mutableStateArray_0[0].write(1, shj_value_1);
/* 044 */           append((shj_mutableStateArray_0[0].getRow()).copy());
/* 045 */
/* 046 */         }
/* 047 */       }
/* 048 */     }
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     while ( inputadapter_input_0.hasNext()) {
/* 054 */       InternalRow inputadapter_row_0 = (InternalRow) inputadapter_input_0.next();
/* 055 */
/* 056 */       long inputadapter_value_0 = inputadapter_row_0.getLong(0);
/* 057 */
/* 058 */       shj_doConsume_0(inputadapter_row_0, inputadapter_value_0);
/* 059 */       if (shouldStop()) return;
/* 060 */     }
/* 061 */   }
/* 062 */
/* 063 */ }
```

Broadcast hash join codegen for the same query (for reference here):

```
== Subtree 2 / 2 (maxMethodCodeSize:280; maxConstantPoolSize:218(0.33% used); numInnerClasses:0) ==
*(2) BroadcastHashJoin [k1#2L], [k2#6L], Inner, BuildRight, false
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4194304, step=1, splits=1)
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [id=#22]
   +- *(1) Project [(id#4L * 3) AS k2#6L]
      +- *(1) Range (0, 1398101, step=1, splits=1)

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 org.apache.spark.sql.execution.joins.LongHashedRelation bhj_relation_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 */
/* 032 */     bhj_relation_0 = ((org.apache.spark.sql.execution.joins.LongHashedRelation) ((org.apache.spark.broadcast.TorrentBroadcast) references[1] /* broadcast */).value()).asReadOnlyCopy();
/* 033 */     incPeakExecutionMemory(bhj_relation_0.estimatedSize());
/* 034 */
/* 035 */     range_mutableStateArray_0[3] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 036 */
/* 037 */   }
/* 038 */
/* 039 */   private void initRange(int idx) {
/* 040 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 041 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 042 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(4194304L);
/* 043 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 044 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 045 */     long partitionEnd;
/* 046 */
/* 047 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 048 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 049 */       range_nextIndex_0 = Long.MAX_VALUE;
/* 050 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 051 */       range_nextIndex_0 = Long.MIN_VALUE;
/* 052 */     } else {
/* 053 */       range_nextIndex_0 = st.longValue();
/* 054 */     }
/* 055 */     range_batchEnd_0 = range_nextIndex_0;
/* 056 */
/* 057 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 058 */     .multiply(step).add(start);
/* 059 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 060 */       partitionEnd = Long.MAX_VALUE;
/* 061 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 062 */       partitionEnd = Long.MIN_VALUE;
/* 063 */     } else {
/* 064 */       partitionEnd = end.longValue();
/* 065 */     }
/* 066 */
/* 067 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
/* 068 */       java.math.BigInteger.valueOf(range_nextIndex_0));
/* 069 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
/* 070 */     if (range_numElementsTodo_0 < 0) {
/* 071 */       range_numElementsTodo_0 = 0;
/* 072 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
/* 073 */       range_numElementsTodo_0++;
/* 074 */     }
/* 075 */   }
/* 076 */
/* 077 */   private void bhj_doConsume_0(long bhj_expr_0_0) throws java.io.IOException {
/* 078 */     // generate join key for stream side
/* 079 */
/* 080 */     // find matches from HashedRelation
/* 081 */     UnsafeRow bhj_matched_0 = false ? null: (UnsafeRow)bhj_relation_0.getValue(bhj_expr_0_0);
/* 082 */     if (bhj_matched_0 != null) {
/* 083 */       {
/* 084 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[2] /* numOutputRows */).add(1);
/* 085 */
/* 086 */         long bhj_value_2 = bhj_matched_0.getLong(0);
/* 087 */         range_mutableStateArray_0[3].reset();
/* 088 */
/* 089 */         range_mutableStateArray_0[3].write(0, bhj_expr_0_0);
/* 090 */
/* 091 */         range_mutableStateArray_0[3].write(1, bhj_value_2);
/* 092 */         append((range_mutableStateArray_0[3].getRow()));
/* 093 */
/* 094 */       }
/* 095 */     }
/* 096 */
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     // initialize Range
/* 101 */     if (!range_initRange_0) {
/* 102 */       range_initRange_0 = true;
/* 103 */       initRange(partitionIndex);
/* 104 */     }
/* 105 */
/* 106 */     while (true) {
/* 107 */       if (range_nextIndex_0 == range_batchEnd_0) {
/* 108 */         long range_nextBatchTodo_0;
/* 109 */         if (range_numElementsTodo_0 > 1000L) {
/* 110 */           range_nextBatchTodo_0 = 1000L;
/* 111 */           range_numElementsTodo_0 -= 1000L;
/* 112 */         } else {
/* 113 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
/* 114 */           range_numElementsTodo_0 = 0;
/* 115 */           if (range_nextBatchTodo_0 == 0) break;
/* 116 */         }
/* 117 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
/* 118 */       }
/* 119 */
/* 120 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
/* 121 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
/* 122 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
/* 123 */
/* 124 */         bhj_doConsume_0(range_value_0);
/* 125 */
/* 126 */         if (shouldStop()) {
/* 127 */           range_nextIndex_0 = range_value_0 + 1L;
/* 128 */           ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localIdx_0 + 1);
/* 129 */           range_inputMetrics_0.incRecordsRead(range_localIdx_0 + 1);
/* 130 */           return;
/* 131 */         }
/* 132 */
/* 133 */       }
/* 134 */       range_nextIndex_0 = range_batchEnd_0;
/* 135 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
/* 136 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
/* 137 */       range_taskContext_0.killTaskIfInterrupted();
/* 138 */     }
/* 139 */   }
/* 140 */
/* 141 */ }
```

### Why are the changes needed?

Codegen shuffled hash join can help save CPU cost. We added shuffled hash join codegen internally in our fork, and seeing obvious improvement in benchmark compared to current non-codegen code path.

Test example query in [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153), seeing 30% wall clock time improvement compared to existing non-codegen code path:

Enable shuffled hash join code-gen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1358 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2323 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    649            679          43          6.5         154.7       1.0X
shuffle hash join wholestage on                     436            465          45          9.6         103.9       1.5X
```

Disable shuffled hash join codegen:

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join wholestage off
  Stopped after 2 iterations, 1345 ms
  Running case: shuffle hash join wholestage on
  Stopped after 5 iterations, 2967 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join wholestage off                    646            673          37          6.5         154.1       1.0X
shuffle hash join wholestage on                     549            594          47          7.6         130.9       1.2X
```

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

No.

### How was this patch tested?

Added unit test in `WholeStageCodegenSuite`.

Closes #29277 from c21/codegen.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-31 05:51:57 +00:00
Takeshi Yamamuro 30e3042dc5 [SPARK-32488][SQL] Use @parser::members and @lexer::members to avoid generating unused code
### What changes were proposed in this pull request?

This PR aims to update `SqlBse.g4` for avoiding generating unused code.
Currently, ANTLR generates unused methods and variables; `isValidDecimal` and `isHint` are only used in the generated lexer. This PR changed the code to use `parser::members` and `lexer::members` to avoid it.

### Why are the changes needed?

To reduce unnecessary code.

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

No.

### How was this patch tested?

Existing tests.

Closes #29296 from maropu/UpdateSqlBase.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 07:51:27 +00:00
Kent Yao 510a1656e6 [SPARK-32412][SQL] Unify error handling for spark thrift server operations
### What changes were proposed in this pull request?

Log error/warn message only once at the server-side for both sync and async modes

### Why are the changes needed?

In b151194299 we make the error logging for  SparkExecuteStatementOperation with `runInBackground=true` not duplicated, but the operations with runInBackground=false and other metadata operation still will be log twice which happened in the operation's `runInternal` method and ThriftCLIService.

In this PR, I propose to reflect the logic to get a unified error handling approach.

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

Yes, when spark.sql.hive.thriftServer.async=false and people call sync APIs the error message will be logged only once at server-side.
### How was this patch tested?

locally verified the result in target/unit-test.log

add unit tests.

Closes #29204 from yaooqinn/SPARK-32412.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:44:13 +00:00
HyukjinKwon e1d7321034 [SPARK-32478][R][SQL] Error message to show the schema mismatch in gapply with Arrow vectorization
### What changes were proposed in this pull request?

This PR proposes to:

1. Fix the error message when the output schema is misbatched with R DataFrame from the given function. For example,

    ```R
    df <- createDataFrame(list(list(a=1L, b="2")))
    count(gapply(df, "a", function(key, group) { group }, structType("a int, b int")))
    ```

    **Before:**

    ```
    Error in handleErrors(returnStatus, conn) :
      ...
      java.lang.UnsupportedOperationException
	    ...
    ```

    **After:**

    ```
    Error in handleErrors(returnStatus, conn) :
     ...
     java.lang.AssertionError: assertion failed: Invalid schema from gapply: expected IntegerType, IntegerType, got IntegerType, StringType
        ...
    ```

2. Update documentation about the schema matching for `gapply` and `dapply`.

### Why are the changes needed?

To show which schema is not matched, and let users know what's going on.

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

Yes, error message is updated as above, and documentation is updated.

### How was this patch tested?

Manually tested and unitttests were added.

Closes #29283 from HyukjinKwon/r-vectorized-error.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-30 15:16:02 +09:00
Max Gekk 99a855575c [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources
### What changes were proposed in this pull request?
When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception:
```
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema:
```

### Why are the changes needed?
To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`:
```Scala
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`
```

Checking of top-level duplicates was introduced by https://github.com/apache/spark/pull/17758.

### Does this PR introduce _any_ user-facing change?
Yes. For the example from SPARK-32431:

ORC:
```scala
java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc
	at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329)
	at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3
	at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61)
	at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323)
```

JSON:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

Parquet:
```scala
+------------+
|StructColumn|
+------------+
|     [0,, 1]|
+------------+
```

Avro:
```scala
+------------+
|StructColumn|
+------------+
|        [,,]|
+------------+
```

After the changes, Parquet, ORC, JSON and Avro output the same error:
```scala
Found duplicate column(s) in the data schema: `camelcase`;
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`;
	at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51)
	at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67)
```

### How was this patch tested?
Run modified test suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite"
$ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*"
```
and added new UT to `SchemaUtilsSuite`.

Closes #29234 from MaxGekk/nested-case-insensitive-column.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-30 06:05:55 +00:00
Wenchen Fan a025a89f4e [SPARK-32332][SQL] Support columnar exchanges
### What changes were proposed in this pull request?

This PR adds abstract classes for shuffle and broadcast, so that users can provide their columnar implementations.

This PR updates several places to use the abstract exchange classes, and also update `AdaptiveSparkPlanExec` so that the columnar rules can see exchange nodes.

This is an alternative of https://github.com/apache/spark/pull/29134 .
Close https://github.com/apache/spark/pull/29134

### Why are the changes needed?

To allow columnar exchanges.

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

no

### How was this patch tested?

new tests

Closes #29262 from cloud-fan/columnar.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-07-29 14:21:47 -05:00
Max Gekk d897825d2d [SPARK-32346][SQL] Support filters pushdown in Avro datasource
### What changes were proposed in this pull request?
In the PR, I propose to support pushed down filters in Avro datasource V1 and V2.
1. Added new SQL config `spark.sql.avro.filterPushdown.enabled` to control filters pushdown to Avro datasource. It is on by default.
2. Renamed `CSVFilters` to `OrderedFilters`.
3. `OrderedFilters` is used in `AvroFileFormat` (DSv1) and in `AvroPartitionReaderFactory` (DSv2)
4. Modified `AvroDeserializer` to return None from the `deserialize` method when pushdown filters return `false`.

### Why are the changes needed?
The changes improve performance on synthetic benchmarks up to **2** times on JDK 11:
```
OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
Filters pushdown:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
w/o filters                                        9614           9669          54          0.1        9614.1       1.0X
pushdown disabled                                 10077          10141          66          0.1       10077.2       1.0X
w/ filters                                         4681           4713          29          0.2        4681.5       2.1X
```

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

### How was this patch tested?
- Added UT to `AvroCatalystDataConversionSuite` and `AvroSuite`
- Re-running `AvroReadBenchmark` using Amazon EC2:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge (spot instance) |
| AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) |
| Java | OpenJDK8/11 installed by`sudo add-apt-repository ppa:openjdk-r/ppa` & `sudo apt install openjdk-11-jdk`|

and `./dev/run-benchmarks`:
```python
#!/usr/bin/env python3

import os
from sparktestsupport.shellutils import run_cmd

benchmarks = [
  ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark']
]

print('Set SPARK_GENERATE_BENCHMARK_FILES=1')
os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1'

for b in benchmarks:
    print("Run benchmark: %s" % b[1])
    run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])])
```

Closes #29145 from MaxGekk/avro-filters-pushdown.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-07-30 01:37:42 +08:00
LantaoJin 26e6574d58 [SPARK-32283][CORE] Kryo should support multiple user registrators
### What changes were proposed in this pull request?
`spark.kryo.registrator` in 3.0 has a regression problem. From [SPARK-12080](https://issues.apache.org/jira/browse/SPARK-12080), it supports multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it donsn't work in 3.0. Fix it by `toSequence` in `Kryo.scala`

### Why are the changes needed?
In previous Spark version (2.x), it supported multiple user registrators by
```scala
private val userRegistrators = conf.get("spark.kryo.registrator", "")
    .split(',').map(_.trim)
    .filter(!_.isEmpty)
```
But it doesn't work in 3.0. It's should be a regression.

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

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

Closes #29123 from LantaoJin/SPARK-32283.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:58:03 +00:00
Terry Kim 45b7212fd3 [SPARK-32401][SQL] Migrate function related commands to use UnresolvedFunc to resolve function identifier
### What changes were proposed in this pull request?

This PR proposes to migrate the following function related commands to use `UnresolvedFunc` to resolve function identifier:
- DROP FUNCTION
- DESCRIBE FUNCTION
- SHOW FUNCTIONS

`DropFunctionStatement`, `DescribeFunctionStatement` and `ShowFunctionsStatement` logical plans are replaced with `DropFunction`, `DescribeFunction` and `ShowFunctions` logical plans respectively, and each contains `UnresolvedFunc` as its child so that it can be resolved in `Analyzer`.

### Why are the changes needed?

Migrating to the new resolution framework, which resolves `UnresolvedFunc` in `Analyzer`.

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

The message of exception thrown when a catalog is resolved to v2 has been merged to:
`function is only supported in v1 catalog`

Previously, it printed out the command used. E.g.,:
`CREATE FUNCTION is only supported in v1 catalog`

### How was this patch tested?

Updated existing tests.

Closes #29198 from imback82/function_framework.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-29 03:55:48 +00:00
Max Gekk b2180c0950 [SPARK-32471][SQL][DOCS][TESTS][PYTHON][SS] Describe JSON option allowNonNumericNumbers
### What changes were proposed in this pull request?
1. Describe the JSON option `allowNonNumericNumbers` which is used in read
2. Add new test cases for allowed JSON field values: NaN, +INF, +Infinity, Infinity, -INF and -Infinity

### Why are the changes needed?
To improve UX with Spark SQL and to provide users full info about the supported option.

### Does this PR introduce _any_ user-facing change?
Yes, in PySpark.

### How was this patch tested?
Added new test to `JsonParsingOptionsSuite`

Closes #29275 from MaxGekk/allowNonNumericNumbers-doc.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-29 12:14:13 +09:00
Michael Munday a3d80564ad [SPARK-32458][SQL][TESTS] Fix incorrectly sized row value reads
### What changes were proposed in this pull request?
Updates to tests to use correctly sized `getInt` or `getLong` calls.

### Why are the changes needed?
The reads were incorrectly sized (i.e. `putLong` paired with `getInt` and `putInt` paired with `getLong`). This causes test failures on big-endian systems.

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

### How was this patch tested?
Tests were run on a big-endian system (s390x). This change is unlikely to have any practical effect on little-endian systems.

Closes #29258 from mundaym/fix-row.

Authored-by: Michael Munday <mike.munday@ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-28 10:36:20 -07:00
Max Gekk c28da672f8 [SPARK-32382][SQL] Override table renaming in JDBC dialects
### What changes were proposed in this pull request?
Override the default implementation of `JdbcDialect.renameTable()`:
```scala
s"ALTER TABLE $oldTable RENAME TO $newTable"
```
in the following JDBC dialects according to official documentation:
- DB2
- Derby
- MS SQL Server
- Teradata

Other dialects follow the default implementation:
- MySQL: https://dev.mysql.com/doc/refman/8.0/en/alter-table.html
- Oracle: https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/ALTER-TABLE.html#GUID-552E7373-BF93-477D-9DA3-B2C9386F2877
- PostgreSQL: https://www.postgresql.org/docs/12/sql-altertable.html

### Why are the changes needed?
To have correct implementation of table renaming for all supported JDBC dialects.

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

### How was this patch tested?
Manually

Closes #29237 from MaxGekk/jdbc-rename-table.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:34:10 +00:00
yi.wu ca1ecf7f9f [SPARK-32459][SQL] Support WrappedArray as customCollectionCls in MapObjects
### What changes were proposed in this pull request?

This PR supports `WrappedArray` as `customCollectionCls` in `MapObjects`.

### Why are the changes needed?

This helps fix the regression caused by SPARK-31826.  For the following test, it can pass in branch-3.0 but fail in master branch:

```scala
test("WrappedArray") {
    val myUdf = udf((a: WrappedArray[Int]) =>
      WrappedArray.make[Int](Array(a.head + 99)))
    checkAnswer(Seq(Array(1))
      .toDF("col")
      .select(myUdf(Column("col"))),
      Row(ArrayBuffer(100)))
  }
```

In SPARK-31826, we've changed the catalyst-to-scala converter from `CatalystTypeConverters` to `ExpressionEncoder.deserializer`. However, `CatalystTypeConverters` supports `WrappedArray` while `ExpressionEncoder.deserializer` doesn't.

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

No,  SPARK-31826 is merged into master and branch-3.1, which haven't been released.

### How was this patch tested?

Added a new test for `WrappedArray` in `UDFSuite`; Also updated `ObjectExpressionsSuite` for `MapObjects`.

Closes #29261 from Ngone51/fix-wrappedarray.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 12:24:15 +00:00
xuewei.linxuewei 12b9787a7f [SPARK-32290][SQL] SingleColumn Null Aware Anti Join Optimize
### What changes were proposed in this pull request?
Normally, a Null aware anti join will be planed into BroadcastNestedLoopJoin which is very time consuming, for instance, in TPCH Query 16.

```
select
    p_brand,
    p_type,
    p_size,
    count(distinct ps_suppkey) as supplier_cnt
from
    partsupp,
    part
where
    p_partkey = ps_partkey
    and p_brand <> 'Brand#45'
    and p_type not like 'MEDIUM POLISHED%'
    and p_size in (49, 14, 23, 45, 19, 3, 36, 9)
    and ps_suppkey not in (
        select
            s_suppkey
        from
            supplier
        where
            s_comment like '%Customer%Complaints%'
    )
group by
    p_brand,
    p_type,
    p_size
order by
    supplier_cnt desc,
    p_brand,
    p_type,
    p_size
```

In above query, will planed into

LeftAnti
    condition Or((ps_suppkey=s_suppkey), IsNull(ps_suppkey=s_suppkey))

Inside BroadcastNestedLoopJoinExec will perform O(M\*N), BUT if there is only single column in NAAJ, we can always change buildSide into a HashSet, and streamedSide just need to lookup in the HashSet, then the calculation will be optimized into O(M).

But this optimize is only targeting on null aware anti join with single column case, because multi-column support is much more complicated, we might be able to support multi-column in future.
After apply this patch, the TPCH Query 16 performance decrease from 41mins to 30s

The semantic of null-aware anti join is:

![image](https://user-images.githubusercontent.com/17242071/88077041-66a39a00-cbad-11ea-8fb6-c235c4d219b4.png)

### Why are the changes needed?
TPCH is a common benchmark for distributed compute engine, all other 21 Query works fine on Spark, except for Query 16, apply this patch will make Spark more competitive among all these popular engine. BTW, this patch has restricted rules and only apply on NAAJ Single Column case, which is safe enough.

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

### How was this patch tested?
1. SQLQueryTestSuite with NOT IN keyword SQL, add CONFIG_DIM with spark.sql.optimizeNullAwareAntiJoin on and off
2. added case in org.apache.spark.sql.JoinSuite.
3. added case in org.apache.spark.sql.SubquerySuite.
3. Compare performance before and after applying this patch against TPCH Query 16.
4. config combination against e2e test with following

```
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "sspark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "false",
  "spark.sql.codegen.wholeStage" -> "true"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "false"
),
Map(
  "spark.sql.optimizeNullAwareAntiJoin" -> "true",
  "spark.sql.adaptive.enabled" -> "true",
  "spark.sql.codegen.wholeStage" -> "true"
)
```

Closes #29104 from leanken/leanken-SPARK-32290.

Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-28 04:42:15 +00:00
Frank Yin 8323c8eb56 [SPARK-32059][SQL] Allow nested schema pruning thru window/sort plans
### What changes were proposed in this pull request?
This PR is intended to solve schema pruning not working with window functions, as described in SPARK-32059. It also solved schema pruning not working with `Sort`. It also generalizes with `Project->Filter->[any node can be pruned]`.

### Why are the changes needed?
This is needed because of performance issues with nested structures with querying using window functions as well as sorting.

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

### How was this patch tested?
Introduced two tests: 1) optimizer planning level 2) end-to-end tests with SQL queries.

Closes #28898 from frankyin-factual/master.

Authored-by: Frank Yin <frank@factual.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-28 10:00:21 +09:00
HyukjinKwon c1140661bf [SPARK-32443][CORE] Use POSIX-compatible command -v in testCommandAvailable
### What changes were proposed in this pull request?

This PR aims to use `command -v` in non-Window operating systems instead of executing the given command.

### Why are the changes needed?

1. `command` is POSIX-compatible
    - **POSIX.1-2017**:  https://pubs.opengroup.org/onlinepubs/9699919799/utilities/command.html
2. `command` is faster and safer than the direct execution
    - `command` doesn't invoke another process.
```scala
scala> sys.process.Process("ls").run().exitValue()
LICENSE
NOTICE
bin
doc
lib
man
res1: Int = 0
```

3. The existing way behaves inconsistently.
    - `rm` cannot be checked.

**AS-IS**
```scala
scala> sys.process.Process("rm").run().exitValue()
usage: rm [-f | -i] [-dPRrvW] file ...
       unlink file
res0: Int = 64
```

**TO-BE**
```
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.
scala> sys.process.Process(Seq("sh", "-c", s"command -v ls")).run().exitValue()
/bin/ls
val res1: Int = 0
```

4. The existing logic is already broken in Scala 2.13 environment because it hangs like the following.
```scala
$ bin/scala
Welcome to Scala 2.13.3 (OpenJDK 64-Bit Server VM, Java 1.8.0_262).
Type in expressions for evaluation. Or try :help.

scala> sys.process.Process("cat").run().exitValue() // hang here.
```

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

No. Although this is inside `main` source directory, this is used for testing purpose.

```
$ git grep testCommandAvailable | grep -v 'def testCommandAvailable'
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("wc"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(!TestUtils.testCommandAvailable("some_nonexistent_command"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable("cat"))
core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala:    assume(TestUtils.testCommandAvailable(envCommand))
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:  private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec)
sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala:    if (TestUtils.testCommandAvailable(pythonExec)) {
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala:    skip = !TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("python"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala:    assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("echo | sed"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:      assume(TestUtils.testCommandAvailable("/bin/bash"))
```

### How was this patch tested?

- **Scala 2.12**: Pass the Jenkins with the existing tests and one modified test.
- **Scala 2.13**: Do the following manually. It should pass instead of `hang`.
```
$ dev/change-scala-version.sh 2.13
$ build/mvn test -pl core --am -Pscala-2.13 -Dtest=none -DwildcardSuites=org.apache.spark.rdd.PipedRDDSuite
...
Tests: succeeded 12, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #29241 from dongjoon-hyun/SPARK-32443.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-27 12:02:43 -07:00
Kent Yao d315ebf3a7 [SPARK-32424][SQL] Fix silent data change for timestamp parsing if overflow happens
### What changes were proposed in this pull request?

When using `Seconds.toMicros` API to convert epoch seconds to microseconds,

```scala
 /**
     * Equivalent to
     * {link #convert(long, TimeUnit) MICROSECONDS.convert(duration, this)}.
     * param duration the duration
     * return the converted duration,
     * or {code Long.MIN_VALUE} if conversion would negatively
     * overflow, or {code Long.MAX_VALUE} if it would positively overflow.
     */
```
This PR change it to `Math.multiplyExact(epochSeconds, MICROS_PER_SECOND)`

### Why are the changes needed?

fix silent data change between 3.x and 2.x
```
 ~/Downloads/spark/spark-3.1.0-SNAPSHOT-bin-20200722   bin/spark-sql -S -e "select to_timestamp('300000', 'y');"
+294247-01-10 12:00:54.775807
```
```
 kentyaohulk  ~/Downloads/spark/spark-2.4.5-bin-hadoop2.7  bin/spark-sql -S  -e "select to_timestamp('300000', 'y');"
284550-10-19 15:58:1010.448384
```

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

Yes, we will raise `ArithmeticException` instead of giving the wrong answer if overflow.

### How was this patch tested?

add unit test

Closes #29220 from yaooqinn/SPARK-32424.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:03:14 +00:00
Cheng Su 548b7db345 [SPARK-32420][SQL] Add handling for unique key in non-codegen hash join
### What changes were proposed in this pull request?

`HashRelation` has two separate code paths for unique key look up and non-unique key look up E.g. in its subclass [`UnsafeHashedRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala#L144-L177), unique key look up is more efficient as it does not have e.g. extra `Iterator[UnsafeRow].hasNext()/next()` overhead per row.

`BroadcastHashJoinExec` has handled unique key vs non-unique key separately in [code-gen path](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala#L289-L321). But the non-codegen path for broadcast hash join and shuffled hash join do not separate it yet, so adding the support here.

### Why are the changes needed?

Shuffled hash join and non-codegen broadcast hash join still rely on this code path for execution. So this PR will help save CPU for executing this two type of join. Adding codegen for shuffled hash join would be a different topic and I will add it in https://issues.apache.org/jira/browse/SPARK-32421 .

Ran the same query as [`JoinBenchmark`](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala#L153-L167), with enabling and disabling this feature. Verified 20% wall clock time improvement (switch control and test group order as well to verify the improvement to not be the noise).

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4039 ms
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 2898 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ off                707            808          81          5.9         168.6       1.0X
shuffle hash join unique key SHJ on                 547            580          50          7.7         130.4       1.3X
```

```
Running benchmark: shuffle hash join
  Running case: shuffle hash join unique key SHJ on
  Stopped after 5 iterations, 3333 ms
  Running case: shuffle hash join unique key SHJ off
  Stopped after 5 iterations, 4268 ms

Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join unique key SHJ on                 565            667          60          7.4         134.8       1.0X
shuffle hash join unique key SHJ off                774            854          85          5.4         184.4       0.7X
```

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

No.

### How was this patch tested?

* Added test in `OuterJoinSuite` to cover left outer and right outer join.
* Added test in `ExistenceJoinSuite` to cover left semi join, and existence join.
* [Existing `joinSuite` already covered inner join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala#L182)
* [Existing `ExistenceJoinSuite` already covered left anti join, and existence join.](https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala#L228)

Closes #29216 from c21/unique-key.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 17:01:03 +00:00
SaurabhChawla 99f33ec30f [SPARK-32234][FOLLOWUP][SQL] Update the description of utility method
### What changes were proposed in this pull request?
As the part of this PR https://github.com/apache/spark/pull/29045 added the helper method. This PR is the FOLLOWUP PR to update the description of helper method.

### Why are the changes needed?
For better readability and understanding of the code

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

### How was this patch tested?
Since its only change of updating the description , So ran the Spark shell

Closes #29232 from SaurabhChawla100/SPARK-32234-Desc.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 08:14:02 +00:00
Cheng Su 01cf8a4ce8 [SPARK-32383][SQL] Preserve hash join (BHJ and SHJ) stream side ordering
### What changes were proposed in this pull request?

Currently `BroadcastHashJoinExec` and `ShuffledHashJoinExec` do not preserve children output ordering information (inherit from `SparkPlan.outputOrdering`, which is Nil). This can add unnecessary sort in complex queries involved multiple joins.

Example:

```
withSQLConf(
      SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") {
      val df1 = spark.range(100).select($"id".as("k1"))
      val df2 = spark.range(100).select($"id".as("k2"))
      val df3 = spark.range(3).select($"id".as("k3"))
      val df4 = spark.range(100).select($"id".as("k4"))
      val plan = df1.join(df2, $"k1" === $"k2")
        .join(df3, $"k1" === $"k3")
        .join(df4, $"k1" === $"k4")
        .queryExecution
        .executedPlan
}
```

Current physical plan (extra sort on `k1` before top sort merge join):

```
*(9) SortMergeJoin [k1#220L], [k4#232L], Inner
:- *(6) Sort [k1#220L ASC NULLS FIRST], false, 0
:  +- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight
:     :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner
:     :  :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0
:     :  :  +- Exchange hashpartitioning(k1#220L, 5), true, [id=#128]
:     :  :     +- *(1) Project [id#218L AS k1#220L]
:     :  :        +- *(1) Range (0, 100, step=1, splits=2)
:     :  +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0
:     :     +- Exchange hashpartitioning(k2#224L, 5), true, [id=#134]
:     :        +- *(3) Project [id#222L AS k2#224L]
:     :           +- *(3) Range (0, 100, step=1, splits=2)
:     +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#141]
:        +- *(5) Project [id#226L AS k3#228L]
:           +- *(5) Range (0, 3, step=1, splits=2)
+- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k4#232L, 5), true, [id=#148]
      +- *(7) Project [id#230L AS k4#232L]
         +- *(7) Range (0, 100, step=1, splits=2)
```

Ideal physical plan (no extra sort on `k1` before top sort merge join):

```
*(9) SortMergeJoin [k1#220L], [k4#232L], Inner
:- *(6) BroadcastHashJoin [k1#220L], [k3#228L], Inner, BuildRight
:  :- *(6) SortMergeJoin [k1#220L], [k2#224L], Inner
:  :  :- *(2) Sort [k1#220L ASC NULLS FIRST], false, 0
:  :  :  +- Exchange hashpartitioning(k1#220L, 5), true, [id=#127]
:  :  :     +- *(1) Project [id#218L AS k1#220L]
:  :  :        +- *(1) Range (0, 100, step=1, splits=2)
:  :  +- *(4) Sort [k2#224L ASC NULLS FIRST], false, 0
:  :     +- Exchange hashpartitioning(k2#224L, 5), true, [id=#133]
:  :        +- *(3) Project [id#222L AS k2#224L]
:  :           +- *(3) Range (0, 100, step=1, splits=2)
:  +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])), [id=#140]
:     +- *(5) Project [id#226L AS k3#228L]
:        +- *(5) Range (0, 3, step=1, splits=2)
+- *(8) Sort [k4#232L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(k4#232L, 5), true, [id=#146]
      +- *(7) Project [id#230L AS k4#232L]
         +- *(7) Range (0, 100, step=1, splits=2)
```

### Why are the changes needed?

To avoid unnecessary sort in query, and it has most impact when users read sorted bucketed table.
Though the unnecessary sort is operating on already sorted data, it would have obvious negative impact on IO and query run time if the data is large and external sorting happens.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29181 from c21/ordering.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-27 04:51:32 +00:00
sychen be9f03dc71 [SPARK-32426][SQL] ui shows sql after variable substitution
### What changes were proposed in this pull request?
When submitting sql with variables, the sql displayed by ui is not replaced by variables.

### Why are the changes needed?
See the final executed sql in ui

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

### How was this patch tested?
manual test

Closes #29221 from cxzl25/SPARK-32426.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 03:30:01 -07:00
HyukjinKwon 277a4063ef [SPARK-32422][SQL][TESTS] Use python3 executable instead of python3.6 in IntegratedUDFTestUtils
### What changes were proposed in this pull request?

This PR uses `python3` instead of `python3.6` executable as a fallback in `IntegratedUDFTestUtils`.

### Why are the changes needed?

Currently, GitHub Actions skips pandas UDFs. Python 3.8 is installed explicitly but somehow `python3.6` looks available in GitHub Actions build environment by default.

```
[info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!!
...
[info] - udf/postgreSQL/udf-select_having.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python3.6]. !!! IGNORED !!!
...
```

It was chosen as `python3.6` for Jenkins to pick one Python explicitly; however, looks we're already using `python3` here and there.

It will also reduce the overhead to fix when we deprecate or drop Python versions.

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

No, dev-only.

### How was this patch tested?

It should be tested in Jenkins and GitHub Actions environments here.

Closes #29217 from HyukjinKwon/SPARK-32422.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-25 03:06:45 -07:00
Andy Grove 64a01c0a55 [SPARK-32430][SQL] Extend SparkSessionExtensions to inject rules into AQE query stage preparation
### What changes were proposed in this pull request?

Provide a generic mechanism for plugins to inject rules into the AQE "query prep" stage that happens before query stage creation.

This goes along with https://issues.apache.org/jira/browse/SPARK-32332 where the current AQE implementation doesn't allow for users to properly extend it for columnar processing.

### Why are the changes needed?

The issue here is that we create new query stages but we do not have access to the parent plan of the new query stage so certain things can not be determined because you have to know what the parent did.  With this change it would allow you to add TAGs to be able to figure out what is going on.

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

No.

### How was this patch tested?

A new unit test is included in the PR.

Closes #29224 from andygrove/insert-aqe-rule.

Authored-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 11:03:57 -07:00
Kent Yao d3596c04b0 [SPARK-32406][SQL] Make RESET syntax support single configuration reset
### What changes were proposed in this pull request?

This PR extends the RESET command to support reset SQL configuration one by one.

### Why are the changes needed?

Currently, the reset command only supports restore all of the runtime configurations to their defaults. In most cases, users do not want this,  but just want to restore one or a small group of settings.
The SET command can work as a workaround for this, but you have to keep the defaults in your mind or by temp variables, which turns out not very convenient to use.

Hive supports this:
https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-BeelineExample

reset <key> | Resets the value of a particular configuration variable (key) to the default value.Note: If you misspell the variable name, Beeline will not show an error.
-- | --

PostgreSQL supports this too

https://www.postgresql.org/docs/9.1/sql-reset.html

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

yes, reset can restore one configuration now
### How was this patch tested?

add new unit tests.

Closes #29202 from yaooqinn/SPARK-32406.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-24 09:13:26 -07:00
Max Gekk 8bc799f920 [SPARK-32375][SQL] Basic functionality of table catalog v2 for JDBC
### What changes were proposed in this pull request?
This PR implements basic functionalities of the `TableCatalog` interface, so that end-users can use the JDBC as a catalog.

### Why are the changes needed?
To have at least one built implementation of Catalog Plugin API available to end users. JDBC is perfectly fit for this.

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

### How was this patch tested?
By new test suite `JDBCTableCatalogSuite`.

Closes #29168 from MaxGekk/jdbc-v2.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-24 14:12:43 +00:00
Liang-Chi Hsieh 84efa04c57 [SPARK-32308][SQL] Move by-name resolution logic of unionByName from API code to analysis phase
### What changes were proposed in this pull request?

Currently the by-name resolution logic of `unionByName` is put in API code. This patch moves the logic to analysis phase.
See https://github.com/apache/spark/pull/28996#discussion_r453460284.

### Why are the changes needed?

Logically we should do resolution in analysis phase. This refactoring cleans up API method and makes consistent resolution.

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

No

### How was this patch tested?

Unit tests.

Closes #29107 from viirya/move-union-by-name.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-24 04:33:18 +00:00
Max Gekk 19e3ed765a [SPARK-32415][SQL][TESTS] Enable tests for JSON option: allowNonNumericNumbers
### What changes were proposed in this pull request?
Enable two tests from `JsonParsingOptionsSuite`:
- `allowNonNumericNumbers off`
- `allowNonNumericNumbers on`

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

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

### How was this patch tested?
By running the enabled tests.

Closes #29207 from MaxGekk/allowNonNumericNumbers-tests.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-24 09:55:36 +09:00
Max Gekk 658e87471c [SPARK-30648][SQL][FOLLOWUP] Refactoring of JsonFilters: move config checking out
### What changes were proposed in this pull request?
Refactoring of `JsonFilters`:
- Add an assert to the `skipRow` method to check the input `index`
- Move checking of the SQL config `spark.sql.json.filterPushdown.enabled` from `JsonFilters` to `JacksonParser`.

### Why are the changes needed?
1. The assert should catch incorrect usage of `JsonFilters`
2. The config checking out of `JsonFilters` makes it consistent with `OrderedFilters` (see https://github.com/apache/spark/pull/29145).
3. `JsonFilters` can be used by other datasource in the future and don't depend from the JSON configs.

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

### How was this patch tested?
By existing tests suites:
```
$ build/sbt "sql/test:testOnly org.apache.spark.sql.execution.datasources.json.*"
$ build/sbt "test:testOnly org.apache.spark.sql.catalyst.json.*"
```

Closes #29206 from MaxGekk/json-filters-pushdown-followup.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-24 09:54:11 +09:00
Sean Owen be2eca22e9 [SPARK-32398][TESTS][CORE][STREAMING][SQL][ML] Update to scalatest 3.2.0 for Scala 2.13.3+
### What changes were proposed in this pull request?

Updates to scalatest 3.2.0. Though it looks large, it is 99% changes to the new location of scalatest classes.

### Why are the changes needed?

3.2.0+ has a fix that is required for Scala 2.13.3+ compatibility.

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

No, only affects tests.

### How was this patch tested?

Existing tests.

Closes #29196 from srowen/SPARK-32398.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 16:20:17 -07:00
Terry Kim 35345e30e5 [SPARK-32374][SQL] Disallow setting properties when creating temporary views
### What changes were proposed in this pull request?

Currently, you can specify properties when creating a temporary view. However, the specified properties are not used and can be misleading.

This PR propose to disallow specifying properties when creating temporary views.

### Why are the changes needed?

To avoid confusion by disallowing specifying unused properties.

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

Yes, now if you create a temporary view with properties, the operation will fail:
```
scala> sql("CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1")
org.apache.spark.sql.catalyst.parser.ParseException:
Operation not allowed: CREATE TEMPORARY VIEW ... TBLPROPERTIES (property_name = property_value, ...)(line 1, pos 0)

== SQL ==
CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1
^^^

```

### How was this patch tested?

Added tests

Closes #29167 from imback82/disable_properties_temp_view.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:32:10 +00:00
yi.wu a8e3de36e7 [SPARK-32280][SPARK-32372][SQL] ResolveReferences.dedupRight should only rewrite attributes for ancestor nodes of the conflict plan
### What changes were proposed in this pull request?

This PR refactors `ResolveReferences.dedupRight` to make sure it only rewrite attributes for ancestor nodes of the conflict plan.

### Why are the changes needed?

This is a bug fix.

```scala
sql("SELECT name, avg(age) as avg_age FROM person GROUP BY name")
  .createOrReplaceTempView("person_a")
sql("SELECT p1.name, p2.avg_age FROM person p1 JOIN person_a p2 ON p1.name = p2.name")
  .createOrReplaceTempView("person_b")
sql("SELECT * FROM person_a UNION SELECT * FROM person_b")
  .createOrReplaceTempView("person_c")
sql("SELECT p1.name, p2.avg_age FROM person_c p1 JOIN person_c p2 ON p1.name = p2.name").show()
```
When executing the above query, we'll hit the error:

```scala
[info]   Failed to analyze query: org.apache.spark.sql.AnalysisException: Resolved attribute(s) avg_age#231 missing from name#223,avg_age#218,id#232,age#234,name#233 in operator !Project [name#233, avg_age#231]. Attribute(s) with the same name appear in the operation: avg_age. Please check if the right attribute(s) are used.;;
...
```

The plan below is the problematic plan which is the right plan of a `Join` operator. And, it has conflict plans comparing to the left plan. In this problematic plan, the first `Aggregate` operator (the one under the first child of `Union`) becomes a conflict plan compares to the left one and has a rewrite attribute pair as  `avg_age#218` -> `avg_age#231`. With the current `dedupRight` logic, we'll first replace this `Aggregate` with a new one, and then rewrites the attribute `avg_age#218` from bottom to up. As you can see, projects with the attribute `avg_age#218` of the second child of the `Union` can also be replaced with `avg_age#231`(That means we also rewrite attributes for non-ancestor plans for the conflict plan). Ideally, the attribute `avg_age#218` in the second `Aggregate` operator (the one under the second child of `Union`) should also be replaced. But it didn't because it's an `Alias` while we only rewrite `Attribute` yet. Therefore, the project above the second `Aggregate` becomes unresolved.

```scala
:

:
+- SubqueryAlias p2
   +- SubqueryAlias person_c
      +- Distinct
         +- Union
            :- Project [name#233, avg_age#231]
            :  +- SubqueryAlias person_a
            :     +- Aggregate [name#233], [name#233, avg(cast(age#234 as bigint)) AS avg_age#231]
            :        +- SubqueryAlias person
            :           +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234]
            :              +- ExternalRDD [obj#165]
            +- Project [name#233 AS name#227, avg_age#231 AS avg_age#228]
               +- Project [name#233, avg_age#231]
                  +- SubqueryAlias person_b
                     +- !Project [name#233, avg_age#231]
                        +- Join Inner, (name#233 = name#223)
                           :- SubqueryAlias p1
                           :  +- SubqueryAlias person
                           :     +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#232, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#233, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#234]
                           :        +- ExternalRDD [obj#165]
                           +- SubqueryAlias p2
                              +- SubqueryAlias person_a
                                 +- Aggregate [name#223], [name#223, avg(cast(age#224 as bigint)) AS avg_age#218]
                                    +- SubqueryAlias person
                                       +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).id AS id#222, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).name, true, false) AS name#223, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$Person, true])).age AS age#224]
                                          +- ExternalRDD [obj#165]
```

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

Yes, users would no longer hit the error after this fix.

### How was this patch tested?

Added test.

Closes #29166 from Ngone51/impr-dedup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:24:47 +00:00
Wenchen Fan aa54dcf193 [SPARK-32251][SQL][TESTS][FOLLOWUP] improve SQL keyword test
### What changes were proposed in this pull request?

Improve the `SQLKeywordSuite` so that:
1. it checks keywords under default mode as well
2. it checks if there are typos in the doc (found one and fixed in this PR)

### Why are the changes needed?

better test coverage

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

no

### How was this patch tested?

N/A

Closes #29200 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 14:02:38 +00:00
Dongjoon Hyun aed8dbab1d [SPARK-32364][SQL][FOLLOWUP] Add toMap to return originalMap and documentation
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/29160. We already removed the indeterministicity. This PR aims the following for the existing code base.
1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

### Why are the changes needed?

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`.

Closes #29191 from dongjoon-hyun/SPARK-32364-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-23 06:28:08 -07:00
LantaoJin 182566bf57 [SPARK-32237][SQL] Resolve hint in CTE
### What changes were proposed in this pull request?
This PR is to move `Substitution` rule before `Hints` rule in `Analyzer` to avoid hint in CTE not working.

### Why are the changes needed?
Below SQL in Spark3.0 will throw AnalysisException, but it works in Spark2.x
```sql
WITH cte AS (SELECT /*+ REPARTITION(3) */ T.id, T.data FROM $t1 T)
SELECT cte.id, cte.data FROM cte
```
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`cte.id`' given input columns: [cte.data, cte.id]; line 3 pos 7;
'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl

'Project ['cte.id, 'cte.data]
+- SubqueryAlias cte
   +- Project [id#21L, data#22]
      +- SubqueryAlias T
         +- SubqueryAlias testcat.ns1.ns2.tbl
            +- RelationV2[id#21L, data#22] testcat.ns1.ns2.tbl
```

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

### How was this patch tested?
Add a unit test

Closes #29062 from LantaoJin/SPARK-32237.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:10:45 +00:00
Takuya UESHIN 46169823c0 [SPARK-30616][SQL][FOLLOW-UP] Use only config key name in the config doc
### What changes were proposed in this pull request?

This is a follow-up of #28852.

This PR to use only config name; otherwise the doc for the config entry shows the entire details of the referring configs.

### Why are the changes needed?

The doc for the newly introduced config entry shows the entire details of the referring configs.

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

The doc for the config entry will show only the referring config keys.

### How was this patch tested?

Existing tests.

Closes #29194 from ueshin/issues/SPARK-30616/fup.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-23 03:07:30 +00:00
Kent Yao b151194299 [SPARK-32392][SQL] Reduce duplicate error log for executing sql statement operation in thrift server
### What changes were proposed in this pull request?

This PR removes the duplicated error log which has been logged in `org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation#execute` but logged again in `runInternal`.

Besides, the log4j configuration for SparkExecuteStatementOperation is turned off because it's not very friendly for Jenkins

### Why are the changes needed?

remove the duplicated error log for better  user experience

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

Yes, less log in thrift server's driver log

### How was this patch tested?

locally verified the result in target/unit-test.log

Closes #29189 from yaooqinn/SPARK-32392.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-23 10:12:13 +09:00
ulysses 184074de22 [SPARK-31999][SQL] Add REFRESH FUNCTION command
### What changes were proposed in this pull request?

In Hive mode, permanent functions are shared with Hive metastore so that functions may be modified by other Hive client. With in long-lived spark scene, it's hard to update the change of function.

Here are 2 reasons:
* Spark cache the function in memory using `FunctionRegistry`.
* User may not know the location or classname of udf when using `replace function`.

Note that we use v2 command code path to add new command.

### Why are the changes needed?

Give a easy way to make spark function registry sync with Hive metastore.
Then we can call
```
refresh function functionName
```

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

Yes, new command.

### How was this patch tested?

New UT.

Closes #28840 from ulysses-you/SPARK-31999.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 19:05:50 +00:00
Dongjoon Hyun cd16a10475 [SPARK-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options
### What changes were proposed in this pull request?

When a user have multiple options like `path`, `paTH`, and `PATH` for the same key `path`, `option/options` is non-deterministic because `extraOptions` is `HashMap`. This PR aims to use `CaseInsensitiveMap` instead of `HashMap` to fix this bug fundamentally.

### Why are the changes needed?

Like the following, DataFrame's `option/options` have been non-deterministic in terms of case-insensitivity because it stores the options at `extraOptions` which is using `HashMap` class.

```scala
spark.read
  .option("paTh", "1")
  .option("PATH", "2")
  .option("Path", "3")
  .option("patH", "4")
  .load("5")
...
org.apache.spark.sql.AnalysisException:
Path does not exist: file:/.../1;
```

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

Yes. However, this is a bug fix for the indeterministic cases.

### How was this patch tested?

Pass the Jenkins or GitHub Action with newly added test cases.

Closes #29160 from dongjoon-hyun/SPARK-32364.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-22 07:58:45 -07:00
Takeshi Yamamuro 04bf3511f1 [SPARK-21117][SQL][FOLLOWUP] Define prettyName for WidthBucket
### What changes were proposed in this pull request?

This PR is to define prettyName for `WidthBucket`.
This comes from the gatorsmile's suggestion: https://github.com/apache/spark/pull/28764#discussion_r457802957

### Why are the changes needed?

For a better name.

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

No.

### How was this patch tested?

Existing tests.

Closes #29183 from maropu/SPARK-21117-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-22 02:51:30 -07:00
Max Gekk feca9edbdd [MINOR][SQL][TESTS] Create tables once in JDBC tests
### What changes were proposed in this pull request?
In PR, I propose to create input tables once before executing tests in `JDBCSuite` and `JdbcRDDSuite`. Currently, the table are created before every test in the test suites.

### Why are the changes needed?
This speed up the test suites up 30-40%.

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

### How was this patch tested?
Run the modified test suites

Closes #29176 from MaxGekk/jdbc-suite-before-all.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 08:32:01 +00:00
Kent Yao 29b7eaa438 [MINOR][SQL] Fix warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys
### What changes were proposed in this pull request?

This PR fixes the warning message for ThriftCLIService.GetCrossReference and GetPrimaryKeys

### Why are the changes needed?

Although we haven't had our own implementation for these thrift APIs, but it still worth logging the right message when people call them wrongly.

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

yes, the driver log for the thrift server will log the right message for the ThriftCLIService.GetCrossReference and GetPrimaryKeys APIs

### How was this patch tested?

passing Jenkins.

Closes #29184 from yaooqinn/minor.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-22 08:15:04 +00:00
Cheng Su 39181ff209 [SPARK-32286][SQL] Coalesce bucketed table for shuffled hash join if applicable
### What changes were proposed in this pull request?
Based on a follow up comment in https://github.com/apache/spark/pull/28123, where we can coalesce buckets for shuffled hash join as well. The note here is we only coalesce the buckets from shuffled hash join stream side (i.e. the side not building hash map), so we don't need to worry about OOM when coalescing multiple buckets in one task for building hash map.

> If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.

Refactor existing physical plan rule `CoalesceBucketsInSortMergeJoin` to `CoalesceBucketsInJoin`, for covering shuffled hash join as well.
Refactor existing unit test `CoalesceBucketsInSortMergeJoinSuite` to `CoalesceBucketsInJoinSuite`, for covering shuffled hash join as well.

### Why are the changes needed?
Avoid shuffle for joining different bucketed tables, is also useful for shuffled hash join. In production, we are seeing users to use shuffled hash join to join bucketed tables (set `spark.sql.join.preferSortMergeJoin`=false, to avoid sort), and this can help avoid shuffle if number of buckets are not same.

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

### How was this patch tested?
Added unit tests in `CoalesceBucketsInJoinSuite` for verifying shuffled hash join physical plan.

### Performance number per request from maropu

I was looking at TPCDS per suggestion from maropu. But I found most of queries from TPCDS are doing aggregate, and only several ones are doing join. None of input tables are bucketed. So I took the approach to test a modified version of `TPCDS q93` as

```
SELECT ss_ticket_number, sr_ticket_number
FROM store_sales
JOIN store_returns
ON ss_ticket_number = sr_ticket_number
```

And make `store_sales` and `store_returns` to be bucketed tables.

Physical query plan without coalesce:

```
ShuffledHashJoin [ss_ticket_number#109L], [sr_ticket_number#120L], Inner, BuildLeft
:- Exchange hashpartitioning(ss_ticket_number#109L, 4), true, [id=#67]
:  +- *(1) Project [ss_ticket_number#109L]
:     +- *(1) Filter isnotnull(ss_ticket_number#109L)
:        +- *(1) ColumnarToRow
:           +- FileScan parquet default.store_sales[ss_ticket_number#109L] Batched: true, DataFilters: [isnotnull(ss_ticket_number#109L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_sales], PartitionFilters: [], PushedFilters: [IsNotNull(ss_ticket_number)], ReadSchema: struct<ss_ticket_number:bigint>, SelectedBucketsCount: 2 out of 2
+- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130]
   +- *(2) Filter isnotnull(sr_ticket_number#120L)
      +- *(2) ColumnarToRow
         +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct<sr_returned_date_sk:bigint,sr_return_time_sk:bigint,sr_item_sk:bigint,sr_customer_sk:bigin..., SelectedBucketsCount: 4 out of 4
```

Physical query plan with coalesce:

```
ShuffledHashJoin [ss_ticket_number#109L], [sr_ticket_number#120L], Inner, BuildLeft
:- *(1) Project [ss_ticket_number#109L]
:  +- *(1) Filter isnotnull(ss_ticket_number#109L)
:     +- *(1) ColumnarToRow
:        +- FileScan parquet default.store_sales[ss_ticket_number#109L] Batched: true, DataFilters: [isnotnull(ss_ticket_number#109L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_sales], PartitionFilters: [], PushedFilters: [IsNotNull(ss_ticket_number)], ReadSchema: struct<ss_ticket_number:bigint>, SelectedBucketsCount: 2 out of 2
+- *(2) Project [sr_returned_date_sk#111L, sr_return_time_sk#112L, sr_item_sk#113L, sr_customer_sk#114L, sr_cdemo_sk#115L, sr_hdemo_sk#116L, sr_addr_sk#117L, sr_store_sk#118L, sr_reason_sk#119L, sr_ticket_number#120L, sr_return_quantity#121L, sr_return_amt#122, sr_return_tax#123, sr_return_amt_inc_tax#124, sr_fee#125, sr_return_ship_cost#126, sr_refunded_cash#127, sr_reversed_charge#128, sr_store_credit#129, sr_net_loss#130]
   +- *(2) Filter isnotnull(sr_ticket_number#120L)
      +- *(2) ColumnarToRow
         +- FileScan parquet default.store_returns[sr_returned_date_sk#111L,sr_return_time_sk#112L,sr_item_sk#113L,sr_customer_sk#114L,sr_cdemo_sk#115L,sr_hdemo_sk#116L,sr_addr_sk#117L,sr_store_sk#118L,sr_reason_sk#119L,sr_ticket_number#120L,sr_return_quantity#121L,sr_return_amt#122,sr_return_tax#123,sr_return_amt_inc_tax#124,sr_fee#125,sr_return_ship_cost#126,sr_refunded_cash#127,sr_reversed_charge#128,sr_store_credit#129,sr_net_loss#130] Batched: true, DataFilters: [isnotnull(sr_ticket_number#120L)], Format: Parquet, Location: InMemoryFileIndex[file:/Users/chengsu/spark/spark-warehouse/store_returns], PartitionFilters: [], PushedFilters: [IsNotNull(sr_ticket_number)], ReadSchema: struct<sr_returned_date_sk:bigint,sr_return_time_sk:bigint,sr_item_sk:bigint,sr_customer_sk:bigin..., SelectedBucketsCount: 4 out of 4 (Coalesced to 2)
```

Run time improvement as 50% of wall clock time:

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.15.4
Intel(R) Core(TM) i9-9980HK CPU  2.40GHz
shuffle hash join:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
shuffle hash join coalesce bucket off              1541           1664         106          1.9         535.1       1.0X
shuffle hash join coalesce bucket on               1060           1169          81          2.7         368.1       1.5X
```

Closes #29079 from c21/split-bucket.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-07-22 08:56:26 +09:00
Dongjoon Hyun 8c7d6f9733 [SPARK-32377][SQL] CaseInsensitiveMap should be deterministic for addition
### What changes were proposed in this pull request?

This PR aims to fix `CaseInsensitiveMap` to be deterministic for addition.

### Why are the changes needed?

```scala
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
var m = CaseInsensitiveMap(Map.empty[String, String])
Seq(("paTh", "1"), ("PATH", "2"), ("Path", "3"), ("patH", "4"), ("path", "5")).foreach { kv =>
  m = (m + kv).asInstanceOf[CaseInsensitiveMap[String]]
  println(m.get("path"))
}
```

**BEFORE**
```
Some(1)
Some(2)
Some(3)
Some(4)
Some(1)
```

**AFTER**
```
Some(1)
Some(2)
Some(3)
Some(4)
Some(5)
```

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

Yes, but this is a bug fix on non-deterministic behavior.

### How was this patch tested?

Pass the newly added test case.

Closes #29172 from dongjoon-hyun/SPARK-32377.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 22:20:16 -07:00
LantaoJin 8a1c24bb03 [SPARK-32362][SQL][TEST] AdaptiveQueryExecSuite misses verifying AE results
### What changes were proposed in this pull request?
Verify results for `AdaptiveQueryExecSuite`

### Why are the changes needed?
`AdaptiveQueryExecSuite` misses verifying AE results
```scala
QueryTest.sameRows(result.toSeq, df.collect().toSeq)
```
Even the results are different, no fail.

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

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

Closes #29158 from LantaoJin/SPARK-32362.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-21 12:47:45 +09:00
gengjiaan 02114f96d6 [SPARK-32365][SQL] Add a boundary condition for negative index in regexp_extract
### What changes were proposed in this pull request?
The current implement of regexp_extract will throws a unprocessed exception show below:
SELECT regexp_extract('1a 2b 14m', 'd+' -1)

```
java.lang.IndexOutOfBoundsException: No group -1
java.util.regex.Matcher.group(Matcher.java:538)
org.apache.spark.sql.catalyst.expressions.RegExpExtract.nullSafeEval(regexpExpressions.scala:455)
org.apache.spark.sql.catalyst.expressions.TernaryExpression.eval(Expression.scala:704)
org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52)
org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45)
```

### Why are the changes needed?
Fix a bug `java.lang.IndexOutOfBoundsException: No group -1`

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

### How was this patch tested?
new UT

Closes #29161 from beliefer/regexp_extract-group-not-allow-less-than-zero.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 20:34:51 -07:00
Nik Vanderhoof 7d65caebec [SPARK-32338][SQL] Overload slice to accept Column for start and length
### What changes were proposed in this pull request?

Add an overload for the `slice` function that can accept Columns for the `start` and `length` parameters.

### Why are the changes needed?

This will allow users to take slices of arrays based on the length of the arrays, or via data in other columns.
```scala
df.select(slice(x, 4, size(x) - 4))
```

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

Yes, before the `slice` method would only accept Ints for the start and length parameters, now we can pass in Columns and/or Ints.

### How was this patch tested?

I've extended the existing tests for slice but using combinations of Column and Ints.

Closes #29138 from nvander1/SPARK-32338.

Authored-by: Nik Vanderhoof <nikolasrvanderhoof@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2020-07-20 17:48:07 -07:00
HyukjinKwon 133c5edc80 [SPARK-32368][SQL] pathGlobFilter, recursiveFileLookup and basePath should respect case insensitivity
### What changes were proposed in this pull request?

This PR proposes to make the datasource options at `PartitioningAwareFileIndex` respect case insensitivity consistently:
- `pathGlobFilter`
- `recursiveFileLookup `
- `basePath`

### Why are the changes needed?

To support consistent case insensitivity in datasource options.

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

Yes, now users can also use case insensitive options such as `PathglobFilter`.

### How was this patch tested?

Unittest were added. It reuses existing tests and adds extra clues to make it easier to track when the test is broken.

Closes #29165 from HyukjinKwon/SPARK-32368.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-20 13:56:00 -07:00