Commit graph

29538 commits

Author SHA1 Message Date
Max Gekk 4f1e434ec5 [SPARK-34721][SQL] Support add/subtract of a year-month interval to/from a date
### What changes were proposed in this pull request?
Support `date +/- year-month interval`. In the PR, I propose to re-use existing code from the `AddMonths` expression, and extract it to the common base class `AddMonthsBase`. That base class is used in new expression `DateAddYMInterval` and in the existing one `AddMonths` (the `add_months` function).

### Why are the changes needed?
To conform the ANSI SQL standard which requires to support such operation over dates and intervals:
<img width="811" alt="Screenshot 2021-03-12 at 11 36 14" src="https://user-images.githubusercontent.com/1580697/110914390-5f412480-8327-11eb-9f8b-e92e73c0b9cd.png">

### Does this PR introduce _any_ user-facing change?
Should not since new intervals have not been released yet.

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

Closes #31812 from MaxGekk/date-add-year-month-interval.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 14:35:56 +00:00
Dongjoon Hyun 9a7977933f [SPARK-34724][SQL] Fix Interpreted evaluation by using getMethod instead of getDeclaredMethod
### What changes were proposed in this pull request?

This bug was introduced by SPARK-23583 at Apache Spark 2.4.0.

This PR aims to use `getMethod` instead of `getDeclaredMethod`.
```scala
- obj.getClass.getDeclaredMethod(functionName, argClasses: _*)
+ obj.getClass.getMethod(functionName, argClasses: _*)
```

### Why are the changes needed?

`getDeclaredMethod` does not search the super class's method. To invoke `GenericArrayData.toIntArray`, we need to use `getMethod` because it's declared at the super class `ArrayData`.

```
[info] - encode/decode for array of int: [I74655d03 (interpreted path) *** FAILED *** (14 milliseconds)
[info]   Exception thrown while decoding
[info]   Converted: [0,1000000020,3,0,ffffff850000001f,4]
[info]   Schema: value#680
[info]   root
[info]   -- value: array (nullable = true)
[info]       |-- element: integer (containsNull = false)
[info]
[info]
[info]   Encoder:
[info]   class[value[0]: array<int>] (ExpressionEncoderSuite.scala:578)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
[info]   at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
[info]   at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
[info]   at org.scalatest.Assertions.fail(Assertions.scala:949)
[info]   at org.scalatest.Assertions.fail$(Assertions.scala:945)
[info]   at org.scalatest.funsuite.AnyFunSuite.fail(AnyFunSuite.scala:1563)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:578)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$4(PlanTest.scala:50)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:54)
[info]   at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:38)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.withSQLConf(ExpressionEncoderSuite.scala:118)
[info]   at org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest.$anonfun$test$3(PlanTest.scala:50)
...
[info]   Cause: java.lang.RuntimeException: Error while decoding: java.lang.NoSuchMethodException: org.apache.spark.sql.catalyst.util.GenericArrayData.toIntArray()
[info] mapobjects(lambdavariable(MapObject, IntegerType, false, -1), assertnotnull(lambdavariable(MapObject, IntegerType, false, -1)), input[0, array<int>, true], None).toIntArray
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Deserializer.apply(ExpressionEncoder.scala:186)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$encodeDecodeTest$1(ExpressionEncoderSuite.scala:576)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.verifyNotLeakingReflectionObjects(ExpressionEncoderSuite.scala:656)
[info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite.$anonfun$testAndVerifyNotLeakingReflectionObjects$2(ExpressionEncoderSuite.scala:669)
```

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

This causes a runtime exception when we use the interpreted mode.

### How was this patch tested?

Pass the modified unit test case.

Closes #31816 from dongjoon-hyun/SPARK-34724.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-12 21:30:46 +09:00
Kousuke Saruta 03dd33cc98 [SPARK-25769][SPARK-34636][SPARK-34626][SQL] sql method in UnresolvedAttribute, AttributeReference and Alias don't quote qualified names properly
### What changes were proposed in this pull request?

This PR fixes an issue that `sql` method in the following classes which take qualified names don't quote the qualified names properly.

* UnresolvedAttribute
* AttributeReference
* Alias

One instance caused by this issue is reported in SPARK-34626.
```
UnresolvedAttribute("a" :: "b" :: Nil).sql
`a.b` // expected: `a`.`b`
```
And other instances are like as follows.
```
UnresolvedAttribute("a`b"::"c.d"::Nil).sql
a`b.`c.d` // expected: `a``b`.`c.d`

AttributeReference("a.b", IntegerType)(qualifier = "c.d"::Nil).sql
c.d.`a.b` // expected: `c.d`.`a.b`

Alias(AttributeReference("a", IntegerType)(), "b.c")(qualifier = "d.e"::Nil).sql
`a` AS d.e.`b.c` // expected: `a` AS `d.e`.`b.c`
```

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

New test.

Closes #31754 from sarutak/fix-qualified-names.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-12 02:58:46 +00:00
Max Gekk cebe2be221 [SPARK-34718][SQL] Assign pretty names to YearMonthIntervalType and DayTimeIntervalType
### What changes were proposed in this pull request?
In the PR, I propose to override the `typeName()` method in `YearMonthIntervalType` and `DayTimeIntervalType`, and assign them names according to the ANSI SQL standard:
<img width="836" alt="Screenshot 2021-03-11 at 17 29 04" src="https://user-images.githubusercontent.com/1580697/110802854-a54aa980-828f-11eb-956d-dd4fbf14aa72.png">
but keep the type name as singular according existing naming convention for other types.

### Why are the changes needed?
To improve Spark SQL user experience, and have readable types in error messages.

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

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

Closes #31810 from MaxGekk/interval-types-name.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:55:12 -08:00
Angerszhuuuu badca975af [SPARK-34712][SQL][TESTS] Refactor UT about hive build in version, avoid to change every time when upgrade hive version
### What changes were proposed in this pull request?
Use HiveUtils.buildinHiveVersion to replace correspoding Ut about hive version

### Why are the changes needed?
Refactor UT about hive build in version, avoid to change every time when upgrade hive version

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

### How was this patch tested?
Not need

Closes #31807 from AngersZhuuuu/SPARK-34712.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 12:52:29 -08:00
Wenchen Fan 6a42b633bf [SPARK-34713][SQL] Fix group by CreateStruct with ExtractValue
### What changes were proposed in this pull request?

This is a bug caused by https://issues.apache.org/jira/browse/SPARK-31670 . We remove the `Alias` when resolving column references in grouping expressions, which breaks `ResolveCreateNamedStruct`

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

new tests

Closes #31808 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-11 09:21:58 -08:00
wankunde 60e324aa9f [SPARK-34688][PYTHON] Upgrade to Py4J 0.10.9.2
### What changes were proposed in this pull request?
This PR upgrade Py4J from 0.10.9.1 to 0.10.9.2 that contains some bug fixes and improvements.

* expose shell parameter in Popen inside launch_gateway. ([bartdag/py4j220efc3](220efc3716))
* fixed Flake8 errors ([bartdag/py4j6c6ee9a](6c6ee9aedc))

### Why are the changes needed?
To leverage fixes from the upstream in Py4J.

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

### How was this patch tested?
Jenkins build and GitHub Actions will test it out.

Closes #31796 from wankunde/py4j.

Authored-by: wankunde <wankunde@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-11 09:51:41 -06:00
Max Gekk d7bb327aee [SPARK-34695][SQL] Fix long overflow in conversion of minimum duration to microseconds
### What changes were proposed in this pull request?
In the PR, I propose to especially handle the amount of seconds `-9223372036855` in `IntervalUtils. durationToMicros()`. Starting from the amount (any durations with the second field < `-9223372036855`), input durations cannot fit to `Long` in the conversion to microseconds. For example, the amount of microseconds = `Long.MinValue = -9223372036854775808` can be represented in two forms:
1. seconds = -9223372036854, nanoAdjustment = -775808, or
2. seconds = -9223372036855, nanoAdjustment = +224192

And the method `Duration.ofSeconds()` produces the last form but such form causes overflow while converting `-9223372036855` seconds to microseconds.

In the PR, I propose to convert the second form to the first one if the second field of input duration is equal to `-9223372036855`.

### Why are the changes needed?
The changes fix the issue demonstrated by the code:
```scala
scala> durationToMicros(microsToDuration(Long.MinValue))
java.lang.ArithmeticException: long overflow
  at java.lang.Math.multiplyExact(Math.java:892)
  at org.apache.spark.sql.catalyst.util.IntervalUtils$.durationToMicros(IntervalUtils.scala:782)
  ... 49 elided
```
The `durationToMicros()` method cannot handle valid output of `microsToDuration()`.

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

### How was this patch tested?
By running new UT from `IntervalUtilsSuite`.

Closes #31799 from MaxGekk/fix-min-duration.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:21:15 +00:00
ulysses-you 744a73df9e [SPARK-34538][SQL] Hive Metastore support filter by not-in
### What changes were proposed in this pull request?

Add `Not(In)` and `Not(InSet)` pattern when convert filter to metastore.

### Why are the changes needed?

`NOT IN` is a useful condition to prune partition, it would be better to support it.

Technically, we can convert `c not in(x,y)` to `c != x and c != y`, then push it to metastore.

Avoid metastore overflow and respect the config `spark.sql.hive.metastorePartitionPruningInSetThreshold`, `Not(InSet)` won't push to metastore if it's value exceeds the threshold.

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

No.

### How was this patch tested?

Add test.

Closes #31646 from ulysses-you/SPARK-34538.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 15:19:47 +00:00
Cheng Su da086a8ea3 [SPARK-34702][SQL] Avoid unnecessary code generation in JoinCodegenSupport.genBuildSideVars
### What changes were proposed in this pull request?

As a followup from code review in https://github.com/apache/spark/pull/31736#discussion_r588134104 , for `JoinCodegenSupport.genBuildSideVars`, we only need to generate build side variables with default values for LEFT OUTER and RIGHT OUTER join, but not for other join types (i.e. LEFT SEMI and LEFT ANTI). Create this PR to clean up the code.

In addition, change `BroadcastNestedLoopJoinExec` unit test to cover both whole stage code-gen enabled and disabled. Harden the unit tests to exercise all code paths.

### Why are the changes needed?

Avoid unnecessary code generation.

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

No.

### How was this patch tested?

Existing unit tests.
* BHJ and SHJ inner join is covered in `InnerJoinSuite.scala`
* BHJ and SHJ left outer and right outer join are covered in `OuterJoinSuite.scala`
* BHJ and SHJ left semi, left anti and existence join are covered in `ExistenceJoinSuite.scala`

Closes #31802 from c21/join-codegen-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 13:29:45 +00:00
Kousuke Saruta fa1cf5c207 [SPARK-34697][SQL] Allow DESCRIBE FUNCTION and SHOW FUNCTIONS explain about || (string concatenation operator)
### What changes were proposed in this pull request?

This PR fixes the behavior of `SHOW FUNCTIONS` and `DESCRIBE FUNCTION` for the `||` operator.
The result of `SHOW FUNCTIONS` doesn't contains `||` and `DESCRIBE FUNCTION ||` says `Function: || not found.` even though `||` is supported.

### Why are the changes needed?

It's a bug.

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

No.

### How was this patch tested?

Confirmed manually with the following commands.
```
spark-sql> DESCRIBE FUNCTION ||;
Function: ||
Usage: expr1 || expr2 - Returns the concatenation of `expr1` and `expr2`.

spark-sql> SHOW FUNCTIONS;
!
!=
%

...

|
||
~
```

Closes #31800 from sarutak/fix-describe-concat-pipe.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-11 22:11:26 +09:00
Max Gekk 9d3d25bca4 [SPARK-34677][SQL] Support the +/- operators over ANSI SQL intervals
### What changes were proposed in this pull request?
Extend the `Add`, `Subtract` and `UnaryMinus` expression to support `DayTimeIntervalType` and `YearMonthIntervalType` added by #31614.

Note: the expressions can throw the `overflow` exception independently from the SQL config `spark.sql.ansi.enabled`. In this way, the modified expressions always behave in the ANSI mode for the intervals.

### Why are the changes needed?
To conform to the ANSI SQL standard which defines `-/+` over intervals:
<img width="822" alt="Screenshot 2021-03-09 at 21 59 22" src="https://user-images.githubusercontent.com/1580697/110523128-bd50ea80-8122-11eb-9982-782da0088d27.png">

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

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

Closes #31789 from MaxGekk/add-subtruct-intervals.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 10:08:43 +00:00
Dongjoon Hyun 5c4d8f9538 [SPARK-34696][SQL][TESTS] Fix CodegenInterpretedPlanTest to generate correct test cases
### What changes were proposed in this pull request?

SPARK-23596 added `CodegenInterpretedPlanTest` at Apache Spark 2.4.0 in a wrong way because `withSQLConf` depends on the execution time `SQLConf.get` instead of `test` function declaration time. So, the following code executes the test twice without controlling the `CodegenObjectFactoryMode`. This PR aims to fix it correct and introduce a new function `testFallback`.

```scala
trait CodegenInterpretedPlanTest extends PlanTest {

   override protected def test(
       testName: String,
       testTags: Tag*)(testFun: => Any)(implicit pos: source.Position): Unit = {
     val codegenMode = CodegenObjectFactoryMode.CODEGEN_ONLY.toString
     val interpretedMode = CodegenObjectFactoryMode.NO_CODEGEN.toString

     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) {
       super.test(testName + " (codegen path)", testTags: _*)(testFun)(pos)
     }
     withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) {
       super.test(testName + " (interpreted path)", testTags: _*)(testFun)(pos)
     }
   }
 }
```

### Why are the changes needed?

1. We need to use like the following.
```scala
super.test(testName + " (codegen path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { testFun })(pos)
super.test(testName + " (interpreted path)", testTags: _*)(
   withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) { testFun })(pos)
```

2. After we fix this behavior with the above code, several test cases including SPARK-34596 and SPARK-34607 fail because they didn't work at both `CODEGEN` and `INTERPRETED` mode. Those test cases only work at `FALLBACK` mode. So, inevitably, we need to introduce `testFallback`.

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31766 from dongjoon-hyun/SPARK-34596-SPARK-34607.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:41:49 -08:00
Cheng Su 14ad7afa1a [MINOR][SQL] Remove unnecessary extend from BroadcastHashJoinExec
### What changes were proposed in this pull request?

This is just a minor fix. `HashJoin` already extends `JoinCodegenSupport`. So we don't need `CodegenSupport` here for `BroadcastHashJoinExec`. Submitted separately as a PR here per https://github.com/apache/spark/pull/31802#discussion_r592066686 .

### Why are the changes needed?

Clean up code.

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

No.

### How was this patch tested?

Existing unit tests.

Closes #31805 from c21/bhj-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:38:53 -08:00
Cheng Su 9aa8f06313 [SPARK-34711][SQL][TESTS] Exercise code-gen enable/disable code paths for SHJ in join test suites
### What changes were proposed in this pull request?

Per comment in https://github.com/apache/spark/pull/31802#discussion_r592068440 , we would like to exercise whole stage code-gen enabled and disabled code paths in join unit test suites. This is for better test coverage of shuffled hash join.

### Why are the changes needed?

Better test coverage.

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

No.

### How was this patch tested?

Existing and added unit tests here.

Closes #31806 from c21/test-minor.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 23:34:09 -08:00
Anton Okolnychyi 72263797bc [SPARK-34457][SQL] DataSource V2: Add default null ordering to SortDirection
### What changes were proposed in this pull request?

This PR adds a default null ordering to public `SortDirection` to match the Catalyst behavior.

### Why are the changes needed?

The SQL standard does not define the default null ordering for a sort direction. That's why it is up to a query engine to assign one. We need to standardize this in our public connector expressions to avoid ambiguity. That's why I propose to match the behavior in our Catalyst expressions.

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

Yes, it affects unreleased connector expression API.

### How was this patch tested?

Existing tests.

Closes #31580 from aokolnychyi/spark-34457.

Authored-by: Anton Okolnychyi <aokolnychyi@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:47:31 +00:00
Terry Kim 2a6e68e1f7 [SPARK-34546][SQL] AlterViewAs.query should be analyzed during the analysis phase, and AlterViewAs should invalidate the cache
### What changes were proposed in this pull request?

This PR proposes the following:
   * `AlterViewAs.query` is currently analyzed in the physical operator `AlterViewAsCommand`, but it should be analyzed during the analysis phase.
   *  When `spark.sql.legacy.storeAnalyzedPlanForView` is set to true, store `TermporaryViewRelation` which wraps the analyzed plan, similar to #31273.
   *  Try to uncache the view you are altering.

### Why are the changes needed?

Analyzing a plan should be done in the analysis phase if possible.

Not uncaching the view (existing behavior) seems like a bug since the cache may not be used again.

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

Yes, now the view can be uncached if it's already cached.

### How was this patch tested?

Added new tests around uncaching.

The existing tests such as `SQLViewSuite` should cover the analysis changes.

Closes #31652 from imback82/alter_view_child.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-11 05:31:40 +00:00
Sean Owen 5e120e4651 [SPARK-34507][BUILD] Update scala.version in parent POM when changing Scala version for cross-build
### What changes were proposed in this pull request?

The `change-scala-version.sh` script updates Scala versions across the build for cross-build purposes. It manually changes `scala.binary.version` but not `scala.version`.

### Why are the changes needed?

It seems that this has always been an oversight, and the cross-built builds of Spark have an incorrect scala.version. See 2.4.5's 2.12 POM for example, which shows a Scala 2.11 version.
https://search.maven.org/artifact/org.apache.spark/spark-core_2.12/2.4.5/pom

More comments in the JIRA.

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

Should be a build-only bug fix.

### How was this patch tested?

Existing tests, but really N/A

Closes #31801 from srowen/SPARK-34507.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-11 10:02:24 +09:00
Andy Grove fc182f7e7f [SPARK-34682][SQL] Use PrivateMethodTester instead of reflection
### Why are the changes needed?
SPARK-34682 was merged prematurely. This PR implements feedback from the review. I wasn't sure whether I should create a new JIRA or not.

### Does this PR introduce _any_ user-facing change?
No. Just improves the test.

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

Closes #31798 from andygrove/SPARK-34682-follow-up.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-10 12:08:31 -08:00
Andy Grove fd4843803c [SPARK-34682][SQL] Fix regression in canonicalization error check in CustomShuffleReaderExec
### What changes were proposed in this pull request?
There is a regression in 3.1.1 compared to 3.0.2 when checking for a canonicalized plan when executing CustomShuffleReaderExec.

The regression was caused by the call to `sendDriverMetrics` which happens before the check and will always fail if the plan is canonicalized.

### Why are the changes needed?
This is a regression in a useful error check.

### Does this PR introduce _any_ user-facing change?
No. This is not an error that a user would typically see, as far as I know.

### How was this patch tested?
I tested this change locally by making a distribution from this PR branch. Before fixing the regression I saw:

```
java.util.NoSuchElementException: key not found: numPartitions
```

After fixing this regression I saw:

```
java.lang.IllegalStateException: operating on canonicalized plan
```

Closes #31793 from andygrove/SPARK-34682.

Lead-authored-by: Andy Grove <andygrove73@gmail.com>
Co-authored-by: Andy Grove <andygrove@nvidia.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-10 20:48:00 +09:00
Cheng Su a916690dd9 [SPARK-34681][SQL] Fix bug for full outer shuffled hash join when building left side with non-equal condition
### What changes were proposed in this pull request?

For full outer shuffled hash join with building hash map on left side, and having non-equal condition, the join can produce wrong result.

The root cause is `boundCondition` in `HashJoin.scala` always assumes the left side row is `streamedPlan` and right side row is `buildPlan` ([streamedPlan.output ++ buildPlan.output](https://github.com/apache/spark/blob/branch-3.1/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala#L141)). This is valid assumption, except for full outer + build left case.

The fix is to correct `boundCondition` in `HashJoin.scala` to handle full outer + build left case properly. See reproduce in https://issues.apache.org/jira/browse/SPARK-32399?focusedCommentId=17298414&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17298414 .

### Why are the changes needed?

Fix data correctness bug.

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

No.

### How was this patch tested?

Changed the test in `OuterJoinSuite.scala` to cover full outer shuffled hash join.
Before this change, the unit test `basic full outer join using ShuffledHashJoin` in `OuterJoinSuite.scala` is failed.

Closes #31792 from c21/join-bugfix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 22:55:27 -08:00
Wenchen Fan 48377d5bd9 [SPARK-34676][SQL][TEST] TableCapabilityCheckSuite should not inherit all tests from AnalysisSuite
### What changes were proposed in this pull request?

Fixes a mistake in `TableCapabilityCheckSuite`, which runs some tests repeatedly.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

N/A

Closes #31788 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-09 09:02:31 -08:00
Kousuke Saruta 2fd85174e9 [SPARK-34603][SQL] Support ADD ARCHIVE and LIST ARCHIVES command
### What changes were proposed in this pull request?

This PR adds `ADD ARCHIVE` and `LIST ARCHIVES` commands to SQL and updates relevant documents.
SPARK-33530 added `addArchive` and `listArchives` to `SparkContext` but it's not supported yet to add/list archives with SQL.

### Why are the changes needed?

To complement features.

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

No.

### How was this patch tested?

Added new test and confirmed the generated HTML from the updated documents.

Closes #31721 from sarutak/sql-archive.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-09 21:28:35 +09:00
Terry Kim cf8bbc184b [SPARK-34152][SQL][FOLLOW-UP] Global temp view's identifier should be correctly stored
### What changes were proposed in this pull request?

This PR proposed to fix a bug introduced in #31273 (https://github.com/apache/spark/pull/31273/files#r589494855).
### Why are the changes needed?

This fixes a bug where global temp view's database name was not passed correctly.

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

Yes, now the global temp view's database is correctly stored.

### How was this patch tested?

Added a new test that catches the bug.

Closes #31783 from imback82/SPARK-34152-bug-fix.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 12:08:08 +00:00
Amandeep Sharma a9c11896a5 [SPARK-34649][SQL][DOCS] org.apache.spark.sql.DataFrameNaFunctions.replace() fails for column name having a dot
### What changes were proposed in this pull request?

Use resolved attributes instead of data-frame fields for replacing values.

### Why are the changes needed?

dataframe.na.replace() does not work for column having a dot in the name

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

None

### How was this patch tested?

Added unit tests for the same

Closes #31769 from amandeep-sharma/master.

Authored-by: Amandeep Sharma <happyaman91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:47:01 +00:00
Cheng Su b5b198516c [SPARK-34620][SQL] Code-gen broadcast nested loop join (inner/cross)
### What changes were proposed in this pull request?

`BroadcastNestedLoopJoinExec` does not have code-gen, and we can potentially boost the CPU performance for this operator if we add code-gen for it. https://databricks.com/blog/2017/02/16/processing-trillion-rows-per-second-single-machine-can-nested-loop-joins-fast.html also showed the evidence in one fork.

The codegen for `BroadcastNestedLoopJoinExec` shared some code with `HashJoin`, and the interface `JoinCodegenSupport` is created to hold those common logic. This PR is only supporting inner and cross join. Other join types will be added later in followup PRs.

Example query and generated code:

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

```
== Subtree 2 / 2 (maxMethodCodeSize:282; maxConstantPoolSize:203(0.31% used); numInnerClasses:0) ==
*(2) BroadcastNestedLoopJoin BuildRight, Inner, NOT ((k1#2L + 1) = k2#6L)
:- *(2) Project [id#0L AS k1#2L]
:  +- *(2) Range (0, 4, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode, [id=#22]
   +- *(1) Project [id#4L AS k2#6L]
      +- *(1) Range (0, 3, step=1, splits=2)

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

### Why are the changes needed?

Improve query CPU performance. Added a micro benchmark query in `JoinBenchmark.scala`.
Saw 1x of run time improvement:

```
OpenJDK 64-Bit Server VM 11.0.9+11-LTS on Linux 4.14.219-161.340.amzn2.x86_64
Intel(R) Xeon(R) CPU E5-2670 v2  2.50GHz
broadcast nested loop join:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------------------------------------
broadcast nested loop join wholestage off          62922          63052         184          0.3        3000.3       1.0X
broadcast nested loop join wholestage on           30946          30972          26          0.7        1475.6       2.0X
```

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

No.

### How was this patch tested?

* Added unit test in `WholeStageCodegenSuite.scala`, and existing unit tests for `BroadcastNestedLoopJoinExec`.
* Updated golden files for several TCPDS query plans, as whole stage code-gen for `BroadcastNestedLoopJoinExec` is triggered.
* Updated `JoinBenchmark-jdk11-results.txt ` and `JoinBenchmark-results.txt` with new benchmark result. Followed previous benchmark PRs - https://github.com/apache/spark/pull/27078 and https://github.com/apache/spark/pull/26003 to use same type of machine:

```
Amazon AWS EC2
type: r3.xlarge
region: us-west-2 (Oregon)
OS: Linux
```

Closes #31736 from c21/nested-join-exec.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 11:45:43 +00:00
Takeshi Yamamuro 43b23fd132 [SPARK-33498][SQL][TESTS][FOLLOWUP] Remove SQLConf.withExistingConf in CastSuite
### What changes were proposed in this pull request?

This PR intends to remove unnecessary `SQLConf.withExistingConf` in `CastSuite`; since we've remove `ParVector ` in #31775, we no longer need to copy SQL configs into each thread env.

### Why are the changes needed?

Clean up the code.

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

No.

### How was this patch tested?

Run the existing tests.

Closes #31785 from maropu/UpdateCastSuite.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-08 23:52:43 -08:00
Max Gekk 6a7701a5be [SPARK-34663][SQL][TESTS] Test year-month and day-time intervals in UDF
### What changes were proposed in this pull request?
Added new tests to `UDFSuite` to check `java.time.Period`/`java.time.Duration` in UDF as input parameters as well as UDF results.

### 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 new tests:
```
$ build/sbt "test:testOnly *UDFSuite"
```

Closes #31779 from MaxGekk/interval-udf.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:58:14 +00:00
Max Gekk 4ea27787bf [SPARK-34666][SQL][TESTS] Test DayTimeIntervalType and YearMonthIntervalType as ordered and atomic types
### What changes were proposed in this pull request?
Add `DayTimeIntervalType` and `YearMonthIntervalType` to `DataTypeTestUtils.ordered`/`atomicTypes`, and implement values generation of those types in `LiteralGenerator`/`RandomDataGenerator`. In this way, the types will be tested automatically in:
1. ArithmeticExpressionSuite:
    - "function least"
    - "function greatest"
2. PredicateSuite
    - "BinaryComparison consistency check"
    - "AND, OR, EqualTo, EqualNullSafe consistency check"
3. ConditionalExpressionSuite
    - "if"
4. RandomDataGeneratorSuite
    - "Basic types"
5. CastSuite
    - "null cast"
    - "up-cast"
    - "SPARK-27671: cast from nested null type in struct"
6. OrderingSuite
    - "GenerateOrdering with DayTimeIntervalType"
    - "GenerateOrdering with YearMonthIntervalType"
7. PredicateSuite
    - "IN with different types"
8. UnsafeRowSuite
    - "calling get(ordinal, datatype) on null columns"
9. SortSuite
    - "sorting on YearMonthIntervalType ..."
    - "sorting on DayTimeIntervalType ..."

### 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 affected test suites.

Closes #31782 from MaxGekk/test-interval-as-atomic.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 06:42:59 +00:00
Dongjoon Hyun ba7e525a11 [SPARK-34670][BUILD] Upgrade ZSTD-JNI to 1.4.9-1
### What changes were proposed in this pull request?

This PR aims to upgrade ZSTD-JNI to 1.4.9-1.

### Why are the changes needed?

ZStandard 1.4.9 and its corresponding JNI brings the following bug fixes and improvements.
- https://github.com/facebook/zstd/releases/tag/v1.4.9

One of notable improvement of ZStandard 1.4.9 is `2x faster Long Distance Mode`, but we are not using it yet.

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

No.

### How was this patch tested?

Pass the CIs with the existing tests and there is no regression in ZStandardBenchmark.

Closes #31784 from dongjoon-hyun/ZSTD-149.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-08 22:40:49 -08:00
allisonwang-db c32cac4cd6 [SPARK-34627][SQL] Use FunctionIdentifier in UnresolvedTableValuedFunction
### What changes were proposed in this pull request?
This PR updates UnresolvedTableValuedFunction's name to be a FunctionIdentifier instead of a string.

### Why are the changes needed?
To make UnresolvedTableValuedFunction consistent with UnresolvedFunction that uses FunctionIdentifier as the function name.

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

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

Closes #31749 from allisonwang-db/spark-34627.

Authored-by: allisonwang-db <66282705+allisonwang-db@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-09 05:27:02 +00:00
Gengliang Wang ee756fd695 [SPARK-34665][SQL][DOCS] Revise the type coercion section of ANSI Compliance
### What changes were proposed in this pull request?

1. Fix the table of valid type coercion combinations. Binary type should be allowed casting to String type and disallowed casting to Numeric types.
2. Summary all the `CAST`s that can cause runtime exceptions.

### Why are the changes needed?

Fix a mistake in the docs.

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

No

### How was this patch tested?

Run `jekyll serve` and preview:

![image](https://user-images.githubusercontent.com/1097932/110334374-8fab5a80-7fd7-11eb-86e7-c519cfa41b99.png)

Closes #31781 from gengliangwang/reviseAnsiDoc2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-09 13:19:14 +09:00
yikf f340857757 [SPARK-34541][CORE] Fixed an issue where data could not be cleaned up when unregisterShuffle
### What changes were proposed in this pull request?
Fixed an issue where data could not be cleaned up when unregisterShuffle.

### Why are the changes needed?
While we use the old shuffle fetch protocol, we use partitionId as mapId in the ShuffleBlockId construction,but we use `context.taskAttemptId()` as mapId that it is cached in `taskIdMapsForShuffle` when we `getWriter[K, V]`.

where data could not be cleaned up when unregisterShuffle ,because we remove a shuffle's metadata from the `taskIdMapsForShuffle`'s mapIds, the mapId is `context.taskAttemptId()` instead of partitionId.

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

### How was this patch tested?
add new test.

Closes #31664 from yikf/master.

Authored-by: yikf <13468507104@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-08 07:08:29 -06:00
Swinky 02e74b298a [SPARK-34598][SQL] RewritePredicateSubquery Rule must not update Filters without subqueries
### What changes were proposed in this pull request?
RewritePredicateSubquery Optimizer Rule must not update Filters without subqueries.

Following is one such example.

```
=== Applying Rule org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery ===
 Project [a#0]                                                        Project [a#0]
!+- Filter (((a#0 > 1) OR (b#1 > 2)) AND ((c#2 > 1) AND (d#3 > 2)))   +- Filter ((((a#0 > 1) OR (b#1 > 2)) AND (c#2 > 1)) AND (d#3 > 2))
    +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]                       +- LocalRelation <empty>, [a#0, b#1, c#2, d#3]
```

### Why are the changes needed?
minor change.

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

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

Closes #31712 from Swinky/rewritePredicateFix.

Authored-by: Swinky <mannswinky@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 09:01:49 +00:00
Max Gekk e10bf64769 [SPARK-34615][SQL] Support java.time.Period as an external type of the year-month interval type
### What changes were proposed in this pull request?
In the PR, I propose to extend Spark SQL API to accept [`java.time.Period`](https://docs.oracle.com/javase/8/docs/api/java/time/Period.html) as an external type of recently added new Catalyst type - `YearMonthIntervalType` (see #31614). The Java class `java.time.Period` has similar semantic to ANSI SQL year-month interval type, and it is the most suitable to be an external type for `YearMonthIntervalType`. In more details:
1. Added `PeriodConverter` which converts `java.time.Period` instances to/from internal representation of the Catalyst type `YearMonthIntervalType` (to `Int` type). The `PeriodConverter` object uses new methods of `IntervalUtils`:
    - `periodToMonths()` converts the input period to the total length in months. If this period is too large to fit `Int`, the method throws the exception `ArithmeticException`. **Note:** _the input period has "days" precision, the method just ignores the days unit._
    - `monthToPeriod()` obtains a `java.time.Period` representing a number of months.
2. Support new type `YearMonthIntervalType` in `RowEncoder` via the methods `createDeserializerForPeriod()` and `createSerializerForJavaPeriod()`.
3. Extended the Literal API to construct literals from `java.time.Period` instances.

### Why are the changes needed?
1. To allow users parallelization of `java.time.Period` collections, and construct year-month interval columns. Also to collect such columns back to the driver side.
2. This will allow to write tests in other sub-tasks of SPARK-27790.

### Does this PR introduce _any_ user-facing change?
The PR extends existing functionality. So, users can parallelize instances of the `java.time.Duration` class and collect them back:

```scala
scala> val ds = Seq(java.time.Period.ofYears(10).withMonths(2)).toDS
ds: org.apache.spark.sql.Dataset[java.time.Period] = [value: yearmonthinterval]

scala> ds.collect
res0: Array[java.time.Period] = Array(P10Y2M)
```

### How was this patch tested?
- Added a few tests to `CatalystTypeConvertersSuite` to check conversion from/to `java.time.Period`.
- Checking row encoding by new tests in `RowEncoderSuite`.
- Making literals of `YearMonthIntervalType` are tested in `LiteralExpressionSuite`.
- Check collecting by `DatasetSuite` and `JavaDatasetSuite`.
- New tests in `IntervalUtilsSuites` to check conversions `java.time.Period` <-> months.

Closes #31765 from MaxGekk/java-time-period.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-08 08:33:09 +00:00
yangjie01 43f355b5f2 [SPARK-34597][SQL] Replaces ParquetFileReader.readFooter with ParquetFileReader.open and getFooter
### What changes were proposed in this pull request?
`ParquetFileReader.readFooter` related methods has been identified as `Deprecated` and `Apache Parquet` suggests replace it with the combination of `ParquetFileReader.open() and getFooter()` methods.

This PR introduces the `ParquetFooterReader` utility class due to some repetitive code patterns when read parquet file footer.

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

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

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

Closes #31711 from LuciferYang/parquet-read-footer.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-07 23:38:40 -08:00
HyukjinKwon 2526fdea48 [SPARK-34657][PYTHON][DOCS] Replace the tag of release to the hash to hide RC tags in Binder
### What changes were proposed in this pull request?

Currently Binder link at Spark 3.1.1 (https://mybinder.org/v2/gh/apache/spark/v3.1.1-rc3?filepath=python%2Fdocs%2Fsource%2Fgetting_started%2Fquickstart.ipynb) shows  `v3.1.1-rc3` like:
![Screen Shot 2021-03-08 at 10 10 55 AM](https://user-images.githubusercontent.com/6477701/110262729-ecb70880-7ff7-11eb-92ba-f151d74985a6.png)

After the fix, it will shows the explicit hash:

![Screen Shot 2021-03-08 at 10 17 25 AM](https://user-images.githubusercontent.com/6477701/110262740-f476ad00-7ff7-11eb-8632-5b418ff87024.png)

In addition, this also fixes the examples URL while I am fixing it. For example: https://github.com/apache/spark/tree/v3.1.1-rc3/examples/src/main/python -> https://github.com/apache/spark/tree/1d550c4e902/examples/src/main/python

Note that it is hash in order to make both dev and release easier.

### Why are the changes needed?

To hide RC tags.

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

It will just change the URL shown when Binder is being loaded.

### How was this patch tested?

Manually tested:

```bash
make clean html
```

![Screen Shot 2021-03-08 at 10 17 06 AM](https://user-images.githubusercontent.com/6477701/110262813-2ee04a00-7ff8-11eb-9983-c4484f7832c4.png)

```bash
git_hash=`git rev-parse --short HEAD`
export GIT_HASH=$git_hash
make clean html
```

![Screen Shot 2021-03-08 at 10 17 25 AM](https://user-images.githubusercontent.com/6477701/110262805-2982ff80-7ff8-11eb-8560-e1e2aa7b263a.png)

Closes #31773 from HyukjinKwon/SPARK-34657.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-08 10:48:17 +09:00
Peter Toth ab8a9a0ceb [SPARK-34545][SQL] Fix issues with valueCompare feature of pyrolite
### What changes were proposed in this pull request?

pyrolite 4.21 introduced and enabled value comparison by default (`valueCompare=true`) during object memoization and serialization: https://github.com/irmen/Pyrolite/blob/pyrolite-4.21/java/src/main/java/net/razorvine/pickle/Pickler.java#L112-L122
This change has undesired effect when we serialize a row (actually `GenericRowWithSchema`) to be passed to python: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L60. A simple example is that
```
new GenericRowWithSchema(Array(1.0, 1.0), StructType(Seq(StructField("_1", DoubleType), StructField("_2", DoubleType))))
```
and
```
new GenericRowWithSchema(Array(1, 1), StructType(Seq(StructField("_1", IntegerType), StructField("_2", IntegerType))))
```
are currently equal and the second instance is replaced to the short code of the first one during serialization.

### Why are the changes needed?
The above can cause nasty issues like the one in https://issues.apache.org/jira/browse/SPARK-34545 description:

```
>>> from pyspark.sql.functions import udf
>>> from pyspark.sql.types import *
>>>
>>> def udf1(data_type):
        def u1(e):
            return e[0]
        return udf(u1, data_type)
>>>
>>> df = spark.createDataFrame([((1.0, 1.0), (1, 1))], ['c1', 'c2'])
>>>
>>> df = df.withColumn("c3", udf1(DoubleType())("c1"))
>>> df = df.withColumn("c4", udf1(IntegerType())("c2"))
>>>
>>> df.select("c3").show()
+---+
| c3|
+---+
|1.0|
+---+

>>> df.select("c4").show()
+---+
| c4|
+---+
|  1|
+---+

>>> df.select("c3", "c4").show()
+---+----+
| c3|  c4|
+---+----+
|1.0|null|
+---+----+
```
This is because during serialization from JVM to Python `GenericRowWithSchema(1.0, 1.0)` (`c1`) is memoized first and when `GenericRowWithSchema(1, 1)` (`c2`) comes next, it is replaced to some short code of the `c1` (instead of serializing `c2` out) as they are `equal()`. The python functions then runs but the return type of `c4` is expected to be `IntegerType` and if a different type (`DoubleType`) comes back from python then it is discarded: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L108-L113

After this PR:
```
>>> df.select("c3", "c4").show()
+---+---+
| c3| c4|
+---+---+
|1.0|  1|
+---+---+
```

### Does this PR introduce _any_ user-facing change?
Yes, fixes a correctness issue.

### How was this patch tested?
Added new UT + manual tests.

Closes #31682 from peter-toth/SPARK-34545-fix-row-comparison.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-07 19:12:42 -06:00
wangguangxin.cn 9ec8696f11 [SPARK-34634][SQL] ResolveReferences.dedupRight should handle ScriptTransformation
### What changes were proposed in this pull request?
When we do self join with transform in a CTE, spark will throw AnalysisException.

A simple way to reproduce is

```
create temporary view t as select * from values 0, 1, 2 as t(a);

WITH temp AS (
  SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t
)
SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b
```

before this patch, it throws

```
org.apache.spark.sql.AnalysisException: cannot resolve '`t1.b`' given input columns: [t1.b]; line 6 pos 41;
'Project ['t1.b]
+- 'Join Inner, ('t1.b = 't2.b)
   :- SubqueryAlias t1
   :  +- SubqueryAlias temp
   :     +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),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)
   :        +- SubqueryAlias t
   :           +- Project [a#1]
   :              +- SubqueryAlias t
   :                 +- LocalRelation [a#1]
   +- SubqueryAlias t2
      +- SubqueryAlias temp
         +- ScriptTransformation [a#1], cat, [b#2], ScriptInputOutputSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.DelimitedJSONSerDe),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)
            +- SubqueryAlias t
               +- Project [a#1]
                  +- SubqueryAlias t
                     +- LocalRelation [a#1]
```

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

### How was this patch tested?
Add a UT

Closes #31752 from WangGuangxin/selfjoin-with-transform.

Authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-07 15:53:52 +09:00
Dongjoon Hyun 631a85ed9b [SPARK-34650][BUILD][SS] Exclude zstd-jni transitive dependency from Kafka Client
### What changes were proposed in this pull request?

This PR aims to exclude `zstd-jni` transitive dependency from kafka-client.

### Why are the changes needed?

To prevent future conflicts, the followings are removed. We should use Spark's zstd-jni dependency consistently.

```
$ build/sbt "token-provider-kafka-0-10/dependencyTree" | grep zstd
[info]   | +-com.github.luben:zstd-jni:1.4.4-7

$ build/sbt "streaming-kafka-0-10/dependencyTree" | grep zstd
[info]   | +-com.github.luben:zstd-jni:1.4.4-7
[info]   | | +-com.github.luben:zstd-jni:1.4.4-7

$ build/sbt "sql-kafka-0-10/dependencyTree" | grep zstd
[info]   | +-com.github.luben:zstd-jni:1.4.4-7
[info]   | | +-com.github.luben:zstd-jni:1.4.4-7
```
### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs.

Closes #31767 from dongjoon-hyun/SPARK-34650.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-07 13:53:55 +09:00
Dongjoon Hyun 1f6089b165 [SPARK-34647][CORE] Use ZSTD JNI NoFinalizer classes and bump to 1.4.8-7
### What changes were proposed in this pull request?

This PR aims to use `ZstdInputStreamNoFinalizer` and `ZstdOutputStreamNoFinalizer` classes and upgrade ZSTD JNI to 1.4.8-7.

### Why are the changes needed?

`1.4.8-7` makes `NoFinalizer` classes public again. This improves the performance.
- 57d53a09d2

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

No.

### How was this patch tested?

Pass the CIs.

Closes #31762 from dongjoon-hyun/SPARK-ZSTD-NOFINALIZER.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 10:32:27 -08:00
Yuming Wang 616c818e7c [SPARK-34628][SQL] Remove GlobalLimit operator if its child max rows not larger than limit number
### What changes were proposed in this pull request?

This pr remove `GlobalLimit` operator if its child max rows not larger than limit number. For example:
```
val testRelation = LocalRelation.fromExternalRows(Seq("a".attr.int, "b".attr.int, "c".attr.int), 1.to(10).map(_ => Row(1, 2, 3)) )
val query = GlobalLimit(100, testRelation)
```
We can remove this `GlobalLimit`.

### Why are the changes needed?

Further optimize the query.

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

No.

### How was this patch tested?

Unit test.

Closes #31750 from wangyum/SPARK-34628.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 09:54:15 -08:00
Sean Owen 2f30cdebb1 [SPARK-34642][DOCS][ML] Fix TypeError in Pyspark Linear Regression docs
### What changes were proposed in this pull request?

Fix a call to setParams in the Linear Regression docs example in Pyspark to avoid a TypeError.

### Why are the changes needed?

The example is slightly wrong and we should not show an error in the docs.

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

None

### How was this patch tested?

Existing tests

Closes #31760 from srowen/SPARK-34642.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 07:32:01 -08:00
helloman 1a9722420e [SPARK-34595][SQL] DPP support RLIKE
### What changes were proposed in this pull request?
This pr make DPP support RLIKE expression:

```sql
SELECT date_id, product_id FROM fact_sk f
JOIN dim_store s
ON f.store_id = s.store_id WHERE s.country RLIKE  '[DE|US]'
```
 ### Why are the changes needed?
Improve query performance.

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

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

Closes #31722 from chaojun-zhang/SPARK-34595.

Authored-by: helloman <zcj23085@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-06 20:30:20 +09:00
HyukjinKwon f72b9068ad [SPARK-34643][R][DOCS] Use CRAN URL in canonical form
### What changes were proposed in this pull request?

This PR fixes the URL links to use CRAN URL in canonical form.
CRAN package submission was failed as below:

```
   Found the following (possibly) invalid URLs:
     URL: https://cran.r-project.org/web/packages/e1071/index.html
       From: man/spark.naiveBayes.Rd
       Status: 200
       Message: OK
       CRAN URL not in canonical form
     URL: https://cran.r-project.org/web/packages/mixtools/index.html
       From: man/spark.gaussianMixture.Rd
       Status: 200
       Message: OK
       CRAN URL not in canonical form
     URL: https://cran.r-project.org/web/packages/survival/index.html
       From: man/spark.survreg.Rd
       Status: 200
       Message: OK
       CRAN URL not in canonical form
     URL: https://cran.r-project.org/web/packages/topicmodels/index.html
       From: man/spark.lda.Rd
       Status: 200
       Message: OK
       CRAN URL not in canonical form
     The canonical URL of the CRAN page for a package is
       https://CRAN.R-project.org/package=pkgname
```

### Why are the changes needed?

To fix CRAN package submission

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

It exposes a canoncal form of URLs to end users.

### How was this patch tested?

I manually clicked each links.

Closes #31759 from HyukjinKwon/minor-doc-fixes.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-05 10:08:11 -08:00
Shardul Mahadik 1fd73686ba [SPARK-34624][CORE] Exclude non-jar dependencies of ivy/maven packages
### What changes were proposed in this pull request?
Exclude non-jar dependencies of the ivy/maven packages we want to resolve as our current dependency resolution code assumes artifacts to be jars. 17601e014c/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L1215) and 17601e014c/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L318)

### Why are the changes needed?
Some maven artifacts define non-jar dependencies. One such example is `hive-exec`'s dependency on the `pom` of `apache-curator` https://repo1.maven.org/maven2/org/apache/hive/hive-exec/2.3.8/hive-exec-2.3.8.pom

Today trying to depend on such an artifact using `--packages` will print an error but continue without including the non-jar dependency. Doing the same using `spark.sql("ADD JAR ivy://org.apache.hive:hive-exec:2.3.8?exclude=org.pentaho:pentaho-aggdesigner-algorithm")` will cause a failure. Detailed stacktraces can be found in SPARK-34624.

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

No

### How was this patch tested?

Added unit test. Retried the same example in `spark-shell` which produced the stacktrace in the JIRA.

Closes #31741 from shardulm94/add-jar-filter-poms.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-05 10:05:51 -08:00
Liang-Chi Hsieh c91a756204 [SPARK-34592][WEBUI] Mark indeterminate RDD in Web UI
### What changes were proposed in this pull request?

This patch proposes to mark indeterminate RDD in Web UI.

### Why are the changes needed?

It is somehow hard to track which part is indeterminate in a graph of RDDs. In some cases we may need to track indeterminate RDDs. For example, indeterminate map stage fails and Spark is unable to fallback some parent stages. The developers are usually unable to easily identify indeterminate part from the complicated RDD computation. If Web UI can show up indeterminate RDD like cached RDD, it could be useful to track it.

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

Yes, there is a UI change for users.

### How was this patch tested?

Manual check with Web UI locally. Updated existing unit tests.

<img width="544" alt="Screen Shot 2021-03-02 at 12 38 02 AM" src="https://user-images.githubusercontent.com/68855/109621580-020bce80-7af0-11eb-834f-46b0f89d47c0.png">
<img width="390" alt="Screen Shot 2021-03-05 at 9 27 14 AM" src="https://user-images.githubusercontent.com/68855/110151181-04db1d80-7d95-11eb-8b3a-7235f7fe9eac.png">

Closes #31707 from viirya/SPARK-34592.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-05 09:31:31 -08:00
Angerszhuuuu 654f19dfd7 [SPARK-34621][SQL] Unify output of ShowCreateTableAsSerdeCommand and ShowCreateTableCommand
### What changes were proposed in this pull request?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

### Why are the changes needed?
Unify output of ShowCreateTableAsSerdeCommand  and ShowCreateTableCommand

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

### How was this patch tested?

Closes #31737 from AngersZhuuuu/SPARK-34621.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:44:07 +00:00
suqilong ca326c4bb3 [SPARK-22748][SQL] Analyze __grouping__id as a literal function
### What changes were proposed in this pull request?

This PR intends to refactor the logic to resolve `__grouping_id` in the `Analyzer`; it moves the logic from `ResolveFunctions` to `ResolveReferences` (`resolveLiteralFunction`).

The original author of this PR is sqlwindspeaker (#30781).

Closes #30781.

### Why are the changes needed?

Code refactoring.

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

No.

### How was this patch tested?

Added tests in `AnalysisSuite`.

Closes #31751 from maropu/SPARK-22748.

Authored-by: suqilong <suqilong@qiyi.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-05 07:40:58 +00:00
Gengliang Wang 358697b386 [SPARK-34635][UI] Add trailing slashes in URLs to reduce unnecessary redirects
### What changes were proposed in this pull request?

Add trailing slashes in URLs of Spark UI pages.

### Why are the changes needed?

When a user accesses a URL without a trailing slash, Spark UI always responds with a 302 redirect to a URL with a trailing slash.
![image](https://user-images.githubusercontent.com/1097932/110072744-1be92380-7d33-11eb-98d4-50df12f59ae3.png)

Adding trailing slash to URLs in UI pages can reduce such unnecessary redirects

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

No

### How was this patch tested?

Manual test. It's a very simple change.

Closes #31753 from gengliangwang/reduceRedirect.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-04 23:23:53 -08:00