Commit graph

9740 commits

Author SHA1 Message Date
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
Cheng Su fe07521c9e [SPARK-32330][SQL] Preserve shuffled hash join build side partitioning
### What changes were proposed in this pull request?

Currently `ShuffledHashJoin.outputPartitioning` inherits from `HashJoin.outputPartitioning`, which only preserves stream side partitioning (`HashJoin.scala`):

```
override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning
```

This loses build side partitioning information, and causes extra shuffle if there's another join / group-by after this join.

Example:

```
withSQLConf(
    SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50",
    SQLConf.SHUFFLE_PARTITIONS.key -> "2",
    SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
  val df1 = spark.range(10).select($"id".as("k1"))
  val df2 = spark.range(30).select($"id".as("k2"))
  Seq("inner", "cross").foreach(joinType => {
    val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count()
      .queryExecution.executedPlan
    assert(plan.collect { case _: ShuffledHashJoinExec => true }.size === 1)
    // No extra shuffle before aggregate
    assert(plan.collect { case _: ShuffleExchangeExec => true }.size === 2)
  })
}
```

Current physical plan (having an extra shuffle on `k1` before aggregate)

```
*(4) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L])
+- Exchange hashpartitioning(k1#220L, 2), true, [id=#117]
   +- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L])
      +- *(3) Project [k1#220L]
         +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft
            :- Exchange hashpartitioning(k1#220L, 2), true, [id=#109]
            :  +- *(1) Project [id#218L AS k1#220L]
            :     +- *(1) Range (0, 10, step=1, splits=2)
            +- Exchange hashpartitioning(k2#224L, 2), true, [id=#111]
               +- *(2) Project [id#222L AS k2#224L]
                  +- *(2) Range (0, 30, step=1, splits=2)
```

Ideal physical plan (no shuffle on `k1` before aggregate)

```
*(3) HashAggregate(keys=[k1#220L], functions=[count(1)], output=[k1#220L, count#235L])
+- *(3) HashAggregate(keys=[k1#220L], functions=[partial_count(1)], output=[k1#220L, count#239L])
   +- *(3) Project [k1#220L]
      +- ShuffledHashJoin [k1#220L], [k2#224L], Inner, BuildLeft
         :- Exchange hashpartitioning(k1#220L, 2), true, [id=#107]
         :  +- *(1) Project [id#218L AS k1#220L]
         :     +- *(1) Range (0, 10, step=1, splits=2)
         +- Exchange hashpartitioning(k2#224L, 2), true, [id=#109]
            +- *(2) Project [id#222L AS k2#224L]
               +- *(2) Range (0, 30, step=1, splits=2)
```

This can be fixed by overriding `outputPartitioning` method in `ShuffledHashJoinExec`, similar to `SortMergeJoinExec`.
In addition, also fix one typo in `HashJoin`, as that code path is shared between broadcast hash join and shuffled hash join.

### Why are the changes needed?

To avoid shuffle (for queries having multiple joins or group-by), for saving CPU and IO.

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

No.

### How was this patch tested?

Added unit test in `JoinSuite`.

Closes #29130 from c21/shj.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:38:43 +00:00
Terry Kim e0ecb66f53 [SPARK-31869][SQL] BroadcastHashJoinExec can utilize the build side for its output partitioning
### What changes were proposed in this pull request?

Currently, the `BroadcastHashJoinExec`'s `outputPartitioning` only uses the streamed side's `outputPartitioning`. However, if the join type of `BroadcastHashJoinExec` is an inner-like join, the build side's info (the join keys) can be added to `BroadcastHashJoinExec`'s `outputPartitioning`.

 For example,
```Scala
spark.conf.set("spark.sql.autoBroadcastJoinThreshold", "500")
val t1 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i1", "j1")
val t2 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i2", "j2")
val t3 = (0 until 20).map(i => (i % 7, i % 11)).toDF("i3", "j3")
val t4 = (0 until 100).map(i => (i % 5, i % 13)).toDF("i4", "j4")

// join1 is a sort merge join.
val join1 = t1.join(t2, t1("i1") === t2("i2"))

// join2 is a broadcast join where t3 is broadcasted.
val join2 = join1.join(t3, join1("i1") === t3("i3"))

// Join on the column from the broadcasted side (i3).
val join3 = join2.join(t4, join2("i3") === t4("i4"))

join3.explain
```
You see that `Exchange hashpartitioning(i2#103, 200)` is introduced because there is no output partitioning info from the build side.
```
== Physical Plan ==
*(6) SortMergeJoin [i3#29], [i4#40], Inner
:- *(4) Sort [i3#29 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(i3#29, 200), true, [id=#55]
:     +- *(3) BroadcastHashJoin [i1#7], [i3#29], Inner, BuildRight
:        :- *(3) SortMergeJoin [i1#7], [i2#18], Inner
:        :  :- *(1) Sort [i1#7 ASC NULLS FIRST], false, 0
:        :  :  +- Exchange hashpartitioning(i1#7, 200), true, [id=#28]
:        :  :     +- LocalTableScan [i1#7, j1#8]
:        :  +- *(2) Sort [i2#18 ASC NULLS FIRST], false, 0
:        :     +- Exchange hashpartitioning(i2#18, 200), true, [id=#29]
:        :        +- LocalTableScan [i2#18, j2#19]
:        +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#34]
:           +- LocalTableScan [i3#29, j3#30]
+- *(5) Sort [i4#40 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i4#40, 200), true, [id=#39]
      +- LocalTableScan [i4#40, j4#41]
```
This PR proposes to introduce output partitioning for the build side for `BroadcastHashJoinExec` if the streamed side has a `HashPartitioning` or a collection of `HashPartitioning`s.

There is a new internal config `spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit`, which can limit the number of partitioning a `HashPartitioning` can expand to. It can be set to "0" to disable this feature.

### Why are the changes needed?

To remove unnecessary shuffle.

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

Yes, now the shuffle in the above example can be eliminated:
```
== Physical Plan ==
*(5) SortMergeJoin [i3#108], [i4#119], Inner
:- *(3) Sort [i3#108 ASC NULLS FIRST], false, 0
:  +- *(3) BroadcastHashJoin [i1#86], [i3#108], Inner, BuildRight
:     :- *(3) SortMergeJoin [i1#86], [i2#97], Inner
:     :  :- *(1) Sort [i1#86 ASC NULLS FIRST], false, 0
:     :  :  +- Exchange hashpartitioning(i1#86, 200), true, [id=#120]
:     :  :     +- LocalTableScan [i1#86, j1#87]
:     :  +- *(2) Sort [i2#97 ASC NULLS FIRST], false, 0
:     :     +- Exchange hashpartitioning(i2#97, 200), true, [id=#121]
:     :        +- LocalTableScan [i2#97, j2#98]
:     +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#126]
:        +- LocalTableScan [i3#108, j3#109]
+- *(4) Sort [i4#119 ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(i4#119, 200), true, [id=#130]
      +- LocalTableScan [i4#119, j4#120]
```

### How was this patch tested?

Added new tests.

Closes #28676 from imback82/broadcast_join_output.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:25:51 +00:00
Gengliang Wang d0c83f372b [SPARK-32302][SQL] Partially push down disjunctive predicates through Join/Partitions
### What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/28733 and #28805, CNF conversion is used to push down disjunctive predicates through join and partitions pruning.

It's a good improvement, however, converting all the predicates in CNF can lead to a very long result, even with grouping functions over expressions.  For example, for the following predicate
```
(p0 = '1' AND p1 = '1') OR (p0 = '2' AND p1 = '2') OR (p0 = '3' AND p1 = '3') OR (p0 = '4' AND p1 = '4') OR (p0 = '5' AND p1 = '5') OR (p0 = '6' AND p1 = '6') OR (p0 = '7' AND p1 = '7') OR (p0 = '8' AND p1 = '8') OR (p0 = '9' AND p1 = '9') OR (p0 = '10' AND p1 = '10') OR (p0 = '11' AND p1 = '11') OR (p0 = '12' AND p1 = '12') OR (p0 = '13' AND p1 = '13') OR (p0 = '14' AND p1 = '14') OR (p0 = '15' AND p1 = '15') OR (p0 = '16' AND p1 = '16') OR (p0 = '17' AND p1 = '17') OR (p0 = '18' AND p1 = '18') OR (p0 = '19' AND p1 = '19') OR (p0 = '20' AND p1 = '20')
```
will be converted into a long query(130K characters) in Hive metastore, and there will be error:
```
javax.jdo.JDOException: Exception thrown when executing query : SELECT DISTINCT 'org.apache.hadoop.hive.metastore.model.MPartition' AS NUCLEUS_TYPE,A0.CREATE_TIME,A0.LAST_ACCESS_TIME,A0.PART_NAME,A0.PART_ID,A0.PART_NAME AS NUCORDER0 FROM PARTITIONS A0 LEFT OUTER JOIN TBLS B0 ON A0.TBL_ID = B0.TBL_ID LEFT OUTER JOIN DBS C0 ON B0.DB_ID = C0.DB_ID WHERE B0.TBL_NAME = ? AND C0."NAME" = ? AND ((((((A0.PART_NAME LIKE '%/p1=1' ESCAPE '\' ) OR (A0.PART_NAME LIKE '%/p1=2' ESCAPE '\' )) OR (A0.PART_NAME LIKE '%/p1=3' ESCAPE '\' )) OR ((A0.PART_NAME LIKE '%/p1=4' ESCAPE '\' ) O ...
```

Essentially, we just need to traverse predicate and extract the convertible sub-predicates like what we did in https://github.com/apache/spark/pull/24598. There is no need to maintain the CNF result set.

### Why are the changes needed?

A better implementation for pushing down disjunctive and complex predicates. The pushed down predicates is always equal or shorter than the CNF result.

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

No

### How was this patch tested?

Unit tests

Closes #29101 from gengliangwang/pushJoin.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-20 14:17:31 +00:00
Anton Okolnychyi 0aca1a6ed4 [SPARK-32276][SQL] Remove redundant sorts before repartition nodes
### What changes were proposed in this pull request?

This PR proposes to remove redundant sorts before repartition nodes whenever the data is ordered after the repartitioning.

### Why are the changes needed?

It looks like our `EliminateSorts` rule can be extended further to remove sorts before repartition nodes that don't affect the final output ordering. It seems safe to perform the following rewrites:

- `Sort -> Repartition -> Sort -> Scan` as `Sort -> Repartition -> Scan`
- `Sort -> Repartition -> Project -> Sort -> Scan` as `Sort -> Repartition -> Project -> Scan`

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

No.

### How was this patch tested?

More test cases.

Closes #29089 from aokolnychyi/spark-32276.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-19 12:11:26 -07:00
Takeshi Yamamuro c7a68a920d [SPARK-32344][SQL] Unevaluable expr is set to FIRST/LAST ignoreNullsExpr in distinct aggregates
### What changes were proposed in this pull request?

This PR intends to fix a bug of distinct FIRST/LAST aggregates in v2.4.6/v3.0.0/master;
```
scala> sql("SELECT FIRST(DISTINCT v) FROM VALUES 1, 2, 3 t(v)").show()
...
Caused by: java.lang.UnsupportedOperationException: Cannot evaluate expression: false#37
  at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.eval(Expression.scala:258)
  at org.apache.spark.sql.catalyst.expressions.AttributeReference.eval(namedExpressions.scala:226)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.ignoreNulls(First.scala:68)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions$lzycompute(First.scala:82)
  at org.apache.spark.sql.catalyst.expressions.aggregate.First.updateExpressions(First.scala:81)
  at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$15.apply(HashAggregateExec.scala:268)
```
A root cause of this bug is that the `Aggregation` strategy replaces a foldable boolean `ignoreNullsExpr` expr with a `Unevaluable` expr (`AttributeReference`) for distinct FIRST/LAST aggregate functions. But, this operation cannot be allowed because the `Analyzer` has checked that it must be foldabe;
ffdbbae1d4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala (L74-L76)
So, this PR proposes to change a vriable for `IGNORE NULLS`  from `Expression` to `Boolean` to avoid the case.

### Why are the changes needed?

Bugfix.

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

No.

### How was this patch tested?

Added a test in `DataFrameAggregateSuite`.

Closes #29143 from maropu/SPARK-32344.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-07-19 11:11:42 +09:00
Sean Owen ee624821a9 [SPARK-29292][YARN][K8S][MESOS] Fix Scala 2.13 compilation for remaining modules
### What changes were proposed in this pull request?

See again the related PRs like https://github.com/apache/spark/pull/28971
This completes fixing compilation for 2.13 for all but `repl`, which is a separate task.

### Why are the changes needed?

Eventually, we need to support a Scala 2.13 build, perhaps in Spark 3.1.

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

No

### How was this patch tested?

Existing tests. (2.13 was not tested; this is about getting it to compile without breaking 2.12)

Closes #29147 from srowen/SPARK-29292.4.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-07-18 15:08:00 -07:00
Sudharshann D f9f9309bec [SPARK-31579][SQL] replaced floorDiv to Div
### What changes were proposed in this pull request?

Replaced  floorDiv to just / in `localRebaseGregorianToJulianDays()` in `spark/sql/catalyst/util/RebaseDateTime.scala`

### Why are the changes needed?

Easier to understand the logic/code and a little more efficiency.

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

NO

### How was this patch tested?

Proof of concept [here](https://github.com/apache/spark/pull/28573/files). The operation `utcCal.getTimeInMillis / MILLIS_PER_DAY` results in an interger value already.

Closes #29008 from Sudhar287/SPARK-31579.

Authored-by: Sudharshann D <sudhar287@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-07-18 13:04:58 -05:00
Yaroslav Tkachenko 34baed8139 [SPARK-30616][SQL] Introduce TTL config option for SQL Metadata Cache
### What changes were proposed in this pull request?
New `spark.sql.metadataCacheTTLSeconds` option that adds time-to-live cache behaviour to the existing caches in `FileStatusCache` and `SessionCatalog`.

### Why are the changes needed?
Currently Spark [caches file listing for tables](https://spark.apache.org/docs/2.4.4/sql-data-sources-parquet.html#metadata-refreshing) and requires issuing `REFRESH TABLE` any time the file listing has changed outside of Spark. Unfortunately, simply submitting `REFRESH TABLE` commands could be very cumbersome. Assuming frequently added files, hundreds of tables and dozens of users querying the data (and expecting up-to-date results), manually refreshing metadata for each table is not a solution.

This is a pretty common use-case for streaming ingestion of data, which can be done outside of Spark (with tools like Kafka Connect, etc.).

A similar feature exists in Presto: `hive.file-status-cache-expire-time` can be found [here](https://prestosql.io/docs/current/connector/hive.html#hive-configuration-properties).

### Does this PR introduce _any_ user-facing change?
Yes, it's controlled with the new `spark.sql.metadataCacheTTLSeconds` option.

When it's set to `-1` (by default), the behaviour of caches doesn't change, so it stays _backwards-compatible_.

Otherwise, you can specify a value in seconds, for example `spark.sql.metadataCacheTTLSeconds: 60` means 1-minute cache TTL.

### How was this patch tested?

Added new tests in:

- FileIndexSuite
- SessionCatalogSuite

Closes #28852 from sap1ens/SPARK-30616-metadata-cache-ttl.

Authored-by: Yaroslav Tkachenko <sapiensy@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-07-17 13:40:54 +00:00