Commit graph

31105 commits

Author SHA1 Message Date
Angerszhuuuu 75bffd972d [SPARK-36755][SQL] ArraysOverlap should handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
For query
```
select arrays_overlap(array(cast('nan' as double), 1d), array(cast('nan' as double)))
```
This returns [false], but it should return [true].
This issue is caused by `scala.mutable.HashSet` can't handle `Double.NaN` and `Float.NaN`.

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
arrays_overlap won't handle equal `NaN` value

### How was this patch tested?
Added UT

Closes #34006 from AngersZhuuuu/SPARK-36755.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit b665782f0d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-15 22:32:18 +08:00
Angerszhuuuu e64155691f [SPARK-36702][SQL][FOLLOWUP] ArrayUnion handle duplicated Double.NaN and Float.NaN
### What changes were proposed in this pull request?
According to https://github.com/apache/spark/pull/33955#discussion_r708570515 use normalized  NaN

### Why are the changes needed?
Use normalized NaN for duplicated NaN value

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

### How was this patch tested?
Exiting UT

Closes #34003 from AngersZhuuuu/SPARK-36702-FOLLOWUP.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 638085953f)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-15 22:04:24 +08:00
Dongjoon Hyun 6a1dacb6b6 [SPARK-36712][BUILD][FOLLOWUP] Improve the regex to avoid breaking pom.xml
### What changes were proposed in this pull request?

This PR aims to fix the regex to avoid breaking `pom.xml`.

### Why are the changes needed?

**BEFORE**
```
$ dev/change-scala-version.sh 2.12
$ git diff | head -n10
diff --git a/core/pom.xml b/core/pom.xml
index dbde22f2bf..6ed368353b 100644
--- a/core/pom.xml
+++ b/core/pom.xml
 -35,7 +35,7
   </properties>

   <dependencies>
-    <!--<!--
```

**AFTER**
Since the default Scala version is `2.12`, the following `no-op` is the correct behavior which is consistent with the previous behavior.
```
$ dev/change-scala-version.sh 2.12
$ git diff
```

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

No. This is a dev only change.

### How was this patch tested?

Manually.

Closes #33996 from dongjoon-hyun/SPARK-36712.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit d730ef24fe)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-14 16:27:02 -07:00
Angerszhuuuu a472612eb8 [SPARK-36702][SQL] ArrayUnion handle duplicated Double.NaN and Float.Nan
### What changes were proposed in this pull request?
For query
```
select array_union(array(cast('nan' as double), cast('nan' as double)), array())
```
This returns [NaN, NaN], but it should return [NaN].
This issue is caused by `OpenHashSet` can't handle `Double.NaN` and `Float.NaN` too.
In this pr we add a wrap for OpenHashSet that can handle `null`, `Double.NaN`, `Float.NaN` together

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
ArrayUnion won't show duplicated `NaN` value

### How was this patch tested?
Added UT

Closes #33955 from AngersZhuuuu/SPARK-36702-WrapOpenHashSet.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit f71f37755d)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-14 18:26:02 +08:00
Minchu Yang 4a486f40cf [SPARK-36705][FOLLOW-UP] Fix unnecessary logWarning when PUSH_BASED_SHUFFLE_ENABLED is set to false
### What changes were proposed in this pull request?

Only throw logWarning when `PUSH_BASED_SHUFFLE_ENABLED` is set to true and `canDoPushBasedShuffle` is false

### Why are the changes needed?

Currently, this logWarning will still be printed out even when `PUSH_BASED_SHUFFLE_ENABLED` is set to false, which is unnecessary.

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

No.

### How was this patch tested?

Passed existing UT.

Closes #33984 from rmcyang/SPARK-36705-follow-up.

Authored-by: Minchu Yang <minyang@minyang-mn3.linkedin.biz>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 2d7dc7c7ce)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-09-13 23:24:25 -05:00
Fu Chen 303590b3e9 [SPARK-36715][SQL] InferFiltersFromGenerate should not infer filter for udf
### What changes were proposed in this pull request?

Fix InferFiltersFromGenerate bug, InferFiltersFromGenerate should not infer filter for generate when the children contain an expression which is instance of `org.apache.spark.sql.catalyst.expressions.UserDefinedExpression`.
Before this pr, the following case will throw an exception.

```scala
spark.udf.register("vec", (i: Int) => (0 until i).toArray)
sql("select explode(vec(8)) as c1").show
```

```
Once strategy's idempotence is broken for batch Infer Filters
 GlobalLimit 21                                                        GlobalLimit 21
 +- LocalLimit 21                                                      +- LocalLimit 21
    +- Project [cast(c1#3 as string) AS c1#12]                            +- Project [cast(c1#3 as string) AS c1#12]
       +- Generate explode(vec(8)), false, [c1#3]                            +- Generate explode(vec(8)), false, [c1#3]
          +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))            +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!            +- OneRowRelation                                                     +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!                                                                                     +- OneRowRelation

java.lang.RuntimeException:
Once strategy's idempotence is broken for batch Infer Filters
 GlobalLimit 21                                                        GlobalLimit 21
 +- LocalLimit 21                                                      +- LocalLimit 21
    +- Project [cast(c1#3 as string) AS c1#12]                            +- Project [cast(c1#3 as string) AS c1#12]
       +- Generate explode(vec(8)), false, [c1#3]                            +- Generate explode(vec(8)), false, [c1#3]
          +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))            +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!            +- OneRowRelation                                                     +- Filter ((size(vec(8), true) > 0) AND isnotnull(vec(8)))
!                                                                                     +- OneRowRelation

	at org.apache.spark.sql.errors.QueryExecutionErrors$.onceStrategyIdempotenceIsBrokenForBatchError(QueryExecutionErrors.scala:1200)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.checkBatchIdempotence(RuleExecutor.scala:168)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:254)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200)
	at scala.collection.immutable.List.foreach(List.scala:431)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
	at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:138)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:196)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:196)
	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:134)
	at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:130)
	at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:148)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executedPlan$1(QueryExecution.scala:166)
	at org.apache.spark.sql.execution.QueryExecution.withCteMap(QueryExecution.scala:73)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:163)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:163)
	at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:214)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:259)
	at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:228)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3731)
	at org.apache.spark.sql.Dataset.head(Dataset.scala:2755)
	at org.apache.spark.sql.Dataset.take(Dataset.scala:2962)
	at org.apache.spark.sql.Dataset.getRows(Dataset.scala:288)
	at org.apache.spark.sql.Dataset.showString(Dataset.scala:327)
	at org.apache.spark.sql.Dataset.show(Dataset.scala:807)
```

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

No, only bug fix.

### How was this patch tested?

Unit test.

Closes #33956 from cfmcgrady/SPARK-36715.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 52c5ff20ca)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-14 09:26:21 +09:00
Leona Yoda b3488a50d7 [SPARK-36739][DOCS][PYTHON] Add apache license headers to makefiles
### What changes were proposed in this pull request?

Add apache license headers to makefiles of PySpark documents.

### Why are the changes needed?

Makefiles of PySpark documentations do not have apache license headers, while the other files have.

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

No

### How was this patch tested?

`make html`

Closes #33979 from yoda-mon/add-license-header-makefiles.

Authored-by: Leona Yoda <yodal@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a440025f08)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-14 09:16:39 +09:00
Minchu Yang 79e148ee93 [SPARK-36705][SHUFFLE] Disable push based shuffle when IO encryption is enabled or serializer is not relocatable
### What changes were proposed in this pull request?

Disable push-based shuffle when IO encryption is enabled or serializer does not support relocation of serialized objects.

### Why are the changes needed?

Push based shuffle is not compatible with IO encryption or non-relocatable serialization.

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

No.

### How was this patch tested?

Added some tests to check whether push-based shuffle can be disabled successfully when IO encryption is enabled or a serializer that does not support relocation of serialized object is used.

Closes #33976 from rmcyang/SPARK-36705.

Authored-by: Minchu Yang <minyang@minyang-mn3.linkedin.biz>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 999473b1a5)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-09-13 16:15:42 -05:00
Lukas Rytz 2e7583799e [SPARK-36712][BUILD] Make scala-parallel-collections in 2.13 POM a direct dependency (not in maven profile)
As [reported on `devspark.apache.org`](https://lists.apache.org/thread.html/r84cff66217de438f1389899e6d6891b573780159cd45463acf3657aa%40%3Cdev.spark.apache.org%3E), the published POMs when building with Scala 2.13 have the `scala-parallel-collections` dependency only in the `scala-2.13` profile of the pom.

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

This PR suggests to work around this by un-commenting the `scala-parallel-collections` dependency when switching to 2.13 using the the `change-scala-version.sh` script.

I included an upgrade to scala-parallel-collections version 1.0.3, the changes compared to 0.2.0 are minor.
  - removed OSGi metadata
  - renamed some internal inner classes
  - added `Automatic-Module-Name`

### Why are the changes needed?

According to the posts, this solves issues for developers that write unit tests for their applications.

Stephen Coy suggested to use the https://www.mojohaus.org/flatten-maven-plugin. While this sounds like a more principled solution, it is possibly too risky to do at this specific point in time?

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

No

### How was this patch tested?

Locally

Closes #33948 from lrytz/parCollDep.

Authored-by: Lukas Rytz <lukas.rytz@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit 1a62e6a2c1)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-13 11:06:58 -05:00
Yuto Akutsu b043ee4de7 [SPARK-36738][SQL][DOC] Fixed the wrong documentation on Cot API
### What changes were proposed in this pull request?

Fixed wrong documentation on Cot API

### Why are the changes needed?

[Doc](https://spark.apache.org/docs/latest/api/sql/index.html#cot) says `1/java.lang.Math.cot` but it should be `1/java.lang.Math.tan`.

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

No.

### How was this patch tested?

Manual check.

Closes #33978 from yutoacts/SPARK-36738.

Authored-by: Yuto Akutsu <yuto.akutsu@nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 3747cfdb40)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-13 21:51:43 +09:00
Kousuke Saruta dad566c1f2 [SPARK-36729][BUILD] Upgrade Netty from 4.1.63 to 4.1.68
### What changes were proposed in this pull request?

This PR upgrades Netty from `4.1.63` to `4.1.68`.

All the changes from `4.1.64` to `4.1.68` are as follows.

* 4.1.64 and 4.1.65
  * https://netty.io/news/2021/05/19/4-1-65-Final.html
* 4.1.66
  * https://netty.io/news/2021/07/16/4-1-66-Final.html
* 4.1.67
  * https://netty.io/news/2021/08/16/4-1-67-Final.html
* 4.1.68
  * https://netty.io/news/2021/09/09/4-1-68-Final.html

### Why are the changes needed?

Recently Netty `4.1.68` was released, which includes official M1 Mac support.
* Add support for mac m1
  * https://github.com/netty/netty/pull/11666

`4.1.65` also includes a critical bug fix which Spark might be affected.
* JNI classloader deadlock with latest JDK version
  * https://github.com/netty/netty/issues/11209

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

No.

### How was this patch tested?

CIs.

Closes #33970 from sarutak/upgrade-netty-4.1.68.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit e1e19619b7)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-12 10:07:40 -07:00
yangjie01 90c4cf3cf3 [SPARK-36636][CORE][TEST] LocalSparkCluster change to use tmp workdir in test to avoid directory name collision
### What changes were proposed in this pull request?
As described in SPARK-36636,if the test cases with config `local-cluster[n, c, m]`  are run continuously within 1 second, the workdir name collision will occur because appid use format as `app-yyyyMMddHHmmss-0000` and workdir name associated with it  in test now,  the related logs are as follows:

```
java.io.IOException: Failed to create directory /spark-mine/work/app-20210908074432-0000/1
	at org.apache.spark.deploy.worker.Worker$$anonfun$receive$1.applyOrElse(Worker.scala:578)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:213)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
21/09/08 22:44:32.266 dispatcher-event-loop-0 INFO Worker: Asked to launch executor app-20210908074432-0000/0 for test
21/09/08 22:44:32.266 dispatcher-event-loop-0 ERROR Worker: Failed to launch executor app-20210908074432-0000/0 for test.
java.io.IOException: Failed to create directory /spark-mine/work/app-20210908074432-0000/0
	at org.apache.spark.deploy.worker.Worker$$anonfun$receive$1.applyOrElse(Worker.scala:578)
	at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:213)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75)
	at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

Since the default value of s`park.deploy.maxExecutorRetries` is 10, the test failed will occur when 5 consecutive cases with local-cluster[3, 1, 1024] are completed within 1 second:

1. case 1: use worker directories: `/app-202109102324-0000/0`, `/app-202109102324-0000/1`, `/app-202109102324-0000/2`
2. case 2: retry 3 times then use worker directories: `/app-202109102324-0000/3`, `/app-202109102324-0000/4`, `/app-202109102324-0000/5`
3. case 3: retry 6 times then use worker directories: `/app-202109102324-0000/6`, `/app-202109102324-0000/7`, `/app-202109102324-0000/8`
4. case 4: retry 9 times then use worker directories: `/app-202109102324-0000/9`, `/app-202109102324-0000/10`, `/app-202109102324-0000/11`
5. case 5: retry more than **10** times then **failed**

To avoid this issue, this pr change to use tmp workdir in test with  config `local-cluster[n, c, m]`.

### Why are the changes needed?
Avoid UT failures caused by continuous workdir name collision.

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

### How was this patch tested?

- Pass GA or Jenkins Tests.
- Manual test: `build/mvn clean install -Pscala-2.13 -pl core -am` or `build/mvn clean install -pl core -am`, with Scala 2.13 is easier to reproduce this problem

**Before**

The test failed error logs as follows and randomness in test failure:
```
- SPARK-33084: Add jar support Ivy URI -- test exclude param when transitive=true *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$138(SparkContextSuite.scala:1109)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test different version *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$142(SparkContextSuite.scala:1118)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test invalid param *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$146(SparkContextSuite.scala:1129)
  at org.apache.spark.SparkFunSuite.withLogAppender(SparkFunSuite.scala:235)
  at org.apache.spark.SparkContextSuite.$anonfun$new$145(SparkContextSuite.scala:1127)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  ...
- SPARK-33084: Add jar support Ivy URI -- test multiple transitive params *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$149(SparkContextSuite.scala:1140)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test param key case sensitive *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$154(SparkContextSuite.scala:1155)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  ...
- SPARK-33084: Add jar support Ivy URI -- test transitive value case insensitive *** FAILED ***
  org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.SparkContextSuite.$anonfun$new$134(SparkContextSuite.scala:1101)
org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
org.scalatest.Transformer.apply(Transformer.scala:22)
org.scalatest.Transformer.apply(Transformer.scala:20)
org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:190)
org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:62)
org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:62)
org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
scala.collection.immutable.List.foreach(List.scala:333)
  at org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2647)
  at scala.Option.foreach(Option.scala:437)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2644)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2734)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:95)
  at org.apache.spark.SparkContextSuite.$anonfun$new$159(SparkContextSuite.scala:1166)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)

```

**After**

```
Run completed in 26 minutes, 38 seconds.
Total number of tests run: 2863
Suites: completed 276, aborted 0
Tests: succeeded 2863, failed 0, canceled 4, ignored 8, pending 0
All tests passed.
```

Closes #33963 from LuciferYang/SPARK-36636.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit 0e1157df06)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-12 09:57:18 -05:00
dgd_contributor 6b62459c09 [SPARK-35822][UI] Spark UI-Executor tab is empty in IE11
### What changes were proposed in this pull request?
Refactor some functions in utils.js to fix the empty UI-Executor tab in yarn mode in IE11.

### Why are the changes needed?
Spark UI-Executor tab is empty in IE11: So this PR to fix this.
![Executortab_IE](https://user-images.githubusercontent.com/84778052/132786964-b17b6d12-457f-4ba3-894f-3f2e1c285b1e.PNG)

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

### How was this patch tested?
Existed UT Testcase

Closes #33937 from dgd-contributor/SPARK-35822-v2.

Authored-by: dgd_contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit ebca01f03e)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 15:58:42 -07:00
Kousuke Saruta b8a23e9ccc [SPARK-36725][SQL][TESTS] Ensure HiveThriftServer2Suites to stop Thrift JDBC server on exit
### What changes were proposed in this pull request?

This PR aims to ensure that HiveThriftServer2Suites (e.g. `thriftserver.UISeleniumSuite`) stop Thrift JDBC server on exit using shutdown hook.

### Why are the changes needed?

Normally, HiveThriftServer2Suites stops Thrift JDBC server via `afterAll` method.
But, if they are killed by signal (e.g. Ctrl-C), Thrift JDBC server will be remain.
```
$ jps
2792969 SparkSubmit
```
### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Killed `thriftserver.UISeleniumSuite` by Ctrl-C and confirmed no Thrift JDBC server is remain by jps.

Closes #33967 from sarutak/stop-thrift-on-exit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit c36d70836d)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 15:54:48 -07:00
dgd-contributor d71e2fd1d1 [SPARK-36685][ML][MLLIB] Fix wrong assert messages
### What changes were proposed in this pull request?
Fix wrong assert statement, a mistake when coding

### Why are the changes needed?
 wrong assert statement

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

### How was this patch tested?
 Existing tests

Closes #33953 from dgd-contributor/SPARK-36685.

Authored-by: dgd-contributor <dgd_contributor@viettel.com.vn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 9af0132516)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-11 14:39:55 -07:00
Sean Owen 043dd531a0 [SPARK-36704][CORE] Expand exception handling to more Java 9 cases where reflection is limited at runtime, when reflecting to manage DirectByteBuffer settings
### What changes were proposed in this pull request?

Improve exception handling in the Platform initialization, where it attempts to assess whether reflection is possible to modify DirectByteBuffer. This can apparently fail in more cases on Java 9+ than are currently handled, whereas Spark can continue without reflection if needed.

More detailed comments on the change inline.

### Why are the changes needed?

This exception seems to be possible and fails startup:

```
Caused by: java.lang.reflect.InaccessibleObjectException: Unable to make private java.nio.DirectByteBuffer(long,int) accessible: module java.base does not "opens java.nio" to unnamed module 71e9ddb4
        at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:357)
        at java.base/java.lang.reflect.AccessibleObject.checkCanSetAccessible(AccessibleObject.java:297)
        at java.base/java.lang.reflect.Constructor.checkCanSetAccessible(Constructor.java:188)
        at java.base/java.lang.reflect.Constructor.setAccessible(Constructor.java:181)
        at org.apache.spark.unsafe.Platform.<clinit>(Platform.java:56)
```

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

Should strictly allow Spark to continue in more cases.

### How was this patch tested?

Existing tests.

Closes #33947 from srowen/SPARK-36704.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit e5283f5ed5)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-09-11 13:38:20 -05:00
yi.wu dcb57ab42b Revert "[SPARK-35011][CORE] Avoid Block Manager registrations when StopExecutor msg is in-flight"
This reverts commit b9e53f8937.

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

Revert https://github.com/apache/spark/pull/32114

### Why are the changes needed?

It breaks the expected `BlockManager` re-registration (e.g., heartbeat loss of an active executor) due to deferred removal of `BlockManager`, see the check:
9cefde8db3/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala (L551)

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

No

### How was this patch tested?

Pass existing tests.

Closes #33959 from Ngone51/revert-35011-3.2.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-10 09:29:55 -07:00
Liang-Chi Hsieh b52fbeee2d [SPARK-36669][SQL] Add Lz4 wrappers for Hadoop Lz4 codec
### What changes were proposed in this pull request?

This patch proposes to add a few LZ4 wrapper classes for Parquet Lz4 compression output that uses Hadoop Lz4 codec.

### Why are the changes needed?

Currently we use Hadop 3.3.1's shaded client libraries. Lz4 is a provided dependency in Hadoop Common 3.3.1 for Lz4Codec. But it isn't excluded from relocation in these libraries. So to use lz4 as Parquet codec, we will hit the exception even we include lz4 as dependency.

```
[info]   Cause: java.lang.NoClassDefFoundError: org/apache/hadoop/shaded/net/jpountz/lz4/LZ4Factory
[info]   at org.apache.hadoop.io.compress.lz4.Lz4Compressor.<init>(Lz4Compressor.java:66)
[info]   at org.apache.hadoop.io.compress.Lz4Codec.createCompressor(Lz4Codec.java:119)
[info]   at org.apache.hadoop.io.compress.CodecPool.getCompressor(CodecPool.java:152)
[info]   at org.apache.hadoop.io.compress.CodecPool.getCompressor(CodecPool.java:168)
```

Before the issue is fixed at Hadoop new release, we can add a few wrapper classes for Lz4 codec.

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

No

### How was this patch tested?

Modified test.

Closes #33940 from viirya/lz4-wrappers.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 6bcf330191)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-09 09:31:11 -07:00
Liang-Chi Hsieh d22182e474 [SPARK-34479][SQL][DOC][FOLLOWUP] Add zstandard to avro supported codecs
### What changes were proposed in this pull request?

Adding `zstandard` to avro supported codecs.

### Why are the changes needed?

To improve the document.

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

No

### How was this patch tested?

Doc only.

Closes #33943 from viirya/minor-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 647ffe655f)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-08 23:21:38 -07:00
Liang-Chi Hsieh 2d75b99b97 [SPARK-36670][SQL][TEST][FOLLOWUP] Add AvroCodecSuite
### What changes were proposed in this pull request?

This patch proposes to add also `AvroCodecSuite` as a follow up for SPARK-36670.

### Why are the changes needed?

Improve test coverage.

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

No

### How was this patch tested?

Added test.

Closes #33939 from viirya/SPARK-36670-avro.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit c95d3fe6c9)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-08 22:17:49 -07:00
Andrew Liu 6cb23c163c [SPARK-36686][SQL] Fix SimplifyConditionalsInPredicate to be null-safe
### What changes were proposed in this pull request?

fix SimplifyConditionalsInPredicate to be null-safe

Reproducible:

```
import org.apache.spark.sql.types.{StructField, BooleanType, StructType}
import org.apache.spark.sql.Row

val schema = List(
  StructField("b", BooleanType, true)
)
val data = Seq(
  Row(true),
  Row(false),
  Row(null)
)
val df = spark.createDataFrame(
  spark.sparkContext.parallelize(data),
  StructType(schema)
)

// cartesian product of true / false / null
val df2 = df.select(col("b") as "cond").crossJoin(df.select(col("b") as "falseVal"))
df2.createOrReplaceTempView("df2")

spark.sql("SELECT * FROM df2 WHERE IF(cond, FALSE, falseVal)").show()
// actual:
// +-----+--------+
// | cond|falseVal|
// +-----+--------+
// |false|    true|
// +-----+--------+
spark.sql("SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.SimplifyConditionalsInPredicate")
spark.sql("SELECT * FROM df2 WHERE IF(cond, FALSE, falseVal)").show()
// expected:
// +-----+--------+
// | cond|falseVal|
// +-----+--------+
// |false|    true|
// | null|    true|
// +-----+--------+
```

### Why are the changes needed?

is a regression that leads to incorrect results

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

no

### How was this patch tested?

existing tests

Closes #33928 from hypercubestart/fix-SimplifyConditionalsInPredicate.

Authored-by: Andrew Liu <andrewlliu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 9b633f2075)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-09 11:32:59 +08:00
Xinrong Meng 88bba0c94b [SPARK-36697][PYTHON] Fix dropping all columns of a DataFrame
### What changes were proposed in this pull request?
Fix dropping all columns of a DataFrame

### Why are the changes needed?
When dropping all columns of a pandas-on-Spark DataFrame, a ValueError is raised.
Whereas in pandas, an empty DataFrame reserving the index is returned.
We should follow pandas.

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

From
```py
>>> psdf = ps.DataFrame({"x": [1, 2], "y": [3, 4], "z": [5, 6]})
>>> psdf
   x  y  z
0  1  3  5
1  2  4  6

>>> psdf.drop(['x', 'y', 'z'])
Traceback (most recent call last):
...
ValueError: not enough values to unpack (expected 2, got 0)

```
To
```py
>>> psdf = ps.DataFrame({"x": [1, 2], "y": [3, 4], "z": [5, 6]})
>>> psdf
   x  y  z
0  1  3  5
1  2  4  6

>>> psdf.drop(['x', 'y', 'z'])
Empty DataFrame
Columns: []
Index: [0, 1]
```

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

Closes #33938 from xinrong-databricks/frame_drop_col.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 33bb7b39e9)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-09 09:59:50 +09:00
Huaxin Gao 7e8860751c [SPARK-34952][SQL][FOLLOWUP] Change column type to be NamedReference
### What changes were proposed in this pull request?
Currently, we have `FieldReference` for aggregate column type, should be `NamedReference` instead

### Why are the changes needed?
`FieldReference` is a private class, should use `NamedReference` instead

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

### How was this patch tested?
existing tests

Closes #33927 from huaxingao/agg_followup.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 23794fb303)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-08 14:06:26 +08:00
yangjie01 c4332c7bf0 [SPARK-36684][SQL][TESTS] Add Jackson test dependencies to sql/core module at hadoop-2.7 profile
### What changes were proposed in this pull request?
SPARK-26346 upgrade Parquet related modules from 1.10.1 to 1.11.1 and `parquet-jackson 1.11.1` use `com.fasterxml.jackson` instead of `org.codehaus.jackson`.

So, there are warning logs related to

```
17:12:17.605 WARN org.apache.hadoop.fs.FileSystem: Cannot load filesystem
java.util.ServiceConfigurationError: org.apache.hadoop.fs.FileSystem: Provider org.apache.hadoop.hdfs.web.WebHdfsFileSystem could not be instantiated
...
Caused by: java.lang.ClassNotFoundException: org.codehaus.jackson.map.ObjectMapper
	at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
...
```

when test `sql/core` modules with `hadoop-2.7` profile.

This pr adds test dependencies related to `org.codehaus.jackson` in `sql/core` module when `hadoop-2.7` profile is activated.

### Why are the changes needed?
Clean up test warning logs that shouldn't exist.

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

### How was this patch tested?

- Pass GA or Jenkins Tests.
- Manual test `mvn clean test -pl sql/core -am -DwildcardSuites=none -Phadoop-2.7`

**Before**

No test failed, but warning logs as follows:

```
[INFO] Running test.org.apache.spark.sql.JavaBeanDeserializationSuite
22:42:45.211 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
22:42:46.827 WARN org.apache.hadoop.fs.FileSystem: Cannot load filesystem
java.util.ServiceConfigurationError: org.apache.hadoop.fs.FileSystem: Provider org.apache.hadoop.hdfs.web.WebHdfsFileSystem could not be instantiated
	at java.util.ServiceLoader.fail(ServiceLoader.java:232)
	at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
	at org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2631)
	at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2650)
	at org.apache.hadoop.fs.FsUrlStreamHandlerFactory.<init>(FsUrlStreamHandlerFactory.java:62)
	at org.apache.spark.sql.internal.SharedState$.liftedTree1$1(SharedState.scala:181)
	at org.apache.spark.sql.internal.SharedState$.org$apache$spark$sql$internal$SharedState$$setFsUrlStreamHandlerFactory(SharedState.scala:180)
	at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:54)
	at org.apache.spark.sql.SparkSession.$anonfun$sharedState$1(SparkSession.scala:135)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:135)
	at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:134)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:335)
	at org.apache.spark.sql.test.TestSparkSession.sessionState$lzycompute(TestSQLContext.scala:42)
	at org.apache.spark.sql.test.TestSparkSession.sessionState(TestSQLContext.scala:41)
	at org.apache.spark.sql.SparkSession.$anonfun$new$3(SparkSession.scala:109)
	at scala.Option.map(Option.scala:230)
	at org.apache.spark.sql.SparkSession.$anonfun$new$1(SparkSession.scala:109)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:194)
	at org.apache.spark.sql.types.DataType.sameType(DataType.scala:97)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1(TypeCoercion.scala:291)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.$anonfun$haveSameType$1$adapted(TypeCoercion.scala:291)
	at scala.collection.LinearSeqOptimized.forall(LinearSeqOptimized.scala:85)
	at scala.collection.LinearSeqOptimized.forall$(LinearSeqOptimized.scala:82)
	at scala.collection.immutable.List.forall(List.scala:89)
	at org.apache.spark.sql.catalyst.analysis.TypeCoercion$.haveSameType(TypeCoercion.scala:291)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck(Expression.scala:1074)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataTypeCheck$(Expression.scala:1069)
	at org.apache.spark.sql.catalyst.expressions.If.dataTypeCheck(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType(Expression.scala:1080)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType$(Expression.scala:1079)
	at org.apache.spark.sql.catalyst.expressions.If.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType$lzycompute(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.If.org$apache$spark$sql$catalyst$expressions$ComplexTypeMergingExpression$$internalDataType(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType(Expression.scala:1084)
	at org.apache.spark.sql.catalyst.expressions.ComplexTypeMergingExpression.dataType$(Expression.scala:1084)
	at org.apache.spark.sql.catalyst.expressions.If.dataType(conditionalExpressions.scala:37)
	at org.apache.spark.sql.catalyst.expressions.objects.MapObjects.$anonfun$dataType$4(objects.scala:815)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.catalyst.expressions.objects.MapObjects.dataType(objects.scala:815)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.$anonfun$dataType$9(complexTypeCreator.scala:416)
	at scala.collection.immutable.List.map(List.scala:290)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType$lzycompute(complexTypeCreator.scala:410)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType(complexTypeCreator.scala:409)
	at org.apache.spark.sql.catalyst.expressions.CreateNamedStruct.dataType(complexTypeCreator.scala:398)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.isSerializedAsStruct(ExpressionEncoder.scala:309)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.isSerializedAsStructForTopLevel(ExpressionEncoder.scala:319)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.<init>(ExpressionEncoder.scala:248)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$.javaBean(ExpressionEncoder.scala:75)
	at org.apache.spark.sql.Encoders$.bean(Encoders.scala:154)
	at org.apache.spark.sql.Encoders.bean(Encoders.scala)
	at test.org.apache.spark.sql.JavaBeanDeserializationSuite.testBeanWithArrayFieldDeserialization(JavaBeanDeserializationSuite.java:75)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:364)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:272)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:237)
	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:158)
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:428)
	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:162)
	at org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:562)
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:548)
Caused by: java.lang.NoClassDefFoundError: org/codehaus/jackson/map/ObjectMapper
	at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.<clinit>(WebHdfsFileSystem.java:129)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at java.lang.Class.newInstance(Class.java:442)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
	... 81 more
Caused by: java.lang.ClassNotFoundException: org.codehaus.jackson.map.ObjectMapper
	at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:419)
	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:352)
	... 88 more
```

**After**

There are no more warning logs like above

Closes #33926 from LuciferYang/SPARK-36684.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit acd9c92fa8)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-07 21:40:53 -07:00
itholic 3d50760a3e [SPARK-36531][SPARK-36515][PYTHON] Improve test coverage for data_type_ops/* and groupby
### What changes were proposed in this pull request?

This PR proposes improving test coverage for pandas-on-Spark data types & GroupBy code base, which is written in `data_type_ops/*.py` and `groupby.py` separately.

This PR did the following to improve coverage:
- Add unittest for untested code
- Fix unittest which is not tested properly
- Remove unused code

**NOTE**: This PR is not only include the test-only update, for example it includes the fixing `astype` for binary ops.

pandas-on-Spark Series we have:
```python
>>> psser
0    [49]
1    [50]
2    [51]
dtype: object
```

before:
```python
>>> psser.astype(bool)
Traceback (most recent call last):
...
pyspark.sql.utils.AnalysisException: cannot resolve 'CAST(`0` AS BOOLEAN)' due to data type mismatch: cannot cast binary to boolean;
...
```

after:
```python
>>> psser.astype(bool)
0    True
1    True
2    True
dtype: bool
```

### Why are the changes needed?

To make the project healthier by improving coverage.

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

No.

### How was this patch tested?

Unittest.

Closes #33850 from itholic/SPARK-36531.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 71dbd03fbe)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-08 10:23:06 +09:00
Venkata Sai Akhil Gudesa 4a92b0e278 [SPARK-36677][SQL] NestedColumnAliasing should not push down aggregate functions into projections
### What changes were proposed in this pull request?

This PR filters out `ExtractValues`s that contains any aggregation function in the `NestedColumnAliasing` rule to prevent cases where aggregations are pushed down into projections.

### Why are the changes needed?

To handle a corner/missed case in `NestedColumnAliasing` that can cause users to encounter a runtime exception.

Consider the following schema:
```
root
 |-- a: struct (nullable = true)
 |    |-- c: struct (nullable = true)
 |    |    |-- e: string (nullable = true)
 |    |-- d: integer (nullable = true)
 |-- b: string (nullable = true)
```
and the query:
`SELECT MAX(a).c.e FROM (SELECT a, b FROM test_aggregates) GROUP BY b`

Executing the query before this PR will result in the error:
```
java.lang.UnsupportedOperationException: Cannot generate code for expression: max(input[0, struct<c:struct<e:string>,d:int>, true])
  at org.apache.spark.sql.errors.QueryExecutionErrors$.cannotGenerateCodeForExpressionError(QueryExecutionErrors.scala:83)
  at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode(Expression.scala:312)
  at org.apache.spark.sql.catalyst.expressions.Unevaluable.doGenCode$(Expression.scala:311)
  at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:99)
...
```
The optimised plan before this PR is:

```
'Aggregate [b#1], [_extract_e#5 AS max(a).c.e#3]
+- 'Project [max(a#0).c.e AS _extract_e#5, b#1]
   +- Relation default.test_aggregates[a#0,b#1] parquet
```

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

No

### How was this patch tested?

A new unit test in `NestedColumnAliasingSuite`. The test consists of the repro mentioned earlier.
The produced optimized plan is checked for equivalency with a plan of the form:
```
 Aggregate [b#452], [max(a#451).c.e AS max('a)[c][e]#456]
+- LocalRelation <empty>, [a#451, b#452]
```

Closes #33921 from vicennial/spark-36677.

Authored-by: Venkata Sai Akhil Gudesa <venkata.gudesa@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 2ed6e7bc5d)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 18:16:02 -07:00
Liang-Chi Hsieh e39948fada [SPARK-36670][SQL][TEST] Add FileSourceCodecSuite
### What changes were proposed in this pull request?

This patch mainly proposes to add some e2e test cases in Spark for codec used by main datasources.

### Why are the changes needed?

We found there is no e2e test cases available for main datasources like Parquet, Orc. It makes developers harder to identify possible bugs early. We should add such tests in Spark.

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

No

### How was this patch tested?

Added tests.

Closes #33912 from viirya/SPARK-36670.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 5a0ae694d0)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 16:53:25 -07:00
Andy Grove 533f655690 [SPARK-36666][SQL] Fix regression in AQEShuffleReadExec
Fix regression in AQEShuffleReadExec when used in conjunction with Spark plugins with custom partitioning.

Signed-off-by: Andy Grove <andygrove73gmail.com>

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

Return `UnknownPartitioning` rather than throw an exception in `AQEShuffleReadExec`.

### Why are the changes needed?

The [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) replaces `AQEShuffleReadExec` with a custom operator that runs on the GPU. Due to changes in [SPARK-36315](dd80457ffb), Spark now throws an exception if the shuffle exchange does not have recognized partitioning, and this happens before the postStageOptimizer rules so there is no opportunity to replace this operator now.

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

No.

### How was this patch tested?

I am still in the process of testing this change. I will update the PR in the next few days with status.

Closes #33910 from andygrove/SPARK-36666.

Authored-by: Andy Grove <andygrove73@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit f78d8394dc)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-07 13:50:00 -07:00
Liang-Chi Hsieh acd3856215 [SPARK-36682][CORE][TEST] Add Hadoop sequence file test for different Hadoop codecs
### What changes were proposed in this pull request?

This patch proposes to add e2e tests for using Hadoop codecs to write sequence files.

### Why are the changes needed?

To improve test coverage.

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

No

### How was this patch tested?

Added tests.

Closes #33924 from viirya/hadoop-seq-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 6745d77818)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-07 13:20:09 -07:00
Kousuke Saruta a41dc4516e [SPARK-36153][SQL][DOCS][FOLLOWUP] Fix the description about the possible values of spark.sql.catalogImplementation property
### What changes were proposed in this pull request?

This PR fixes the description about the possible values of `spark.sql.catalogImplementation` property.
It was added in SPARK-36153 (#33362) but the possible values are `hive` or `in-memory` rather than `true` or `false`.

### Why are the changes needed?

To fix wrong description.

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

No.

### How was this patch tested?

I just confirmed `in-memory` and `hive` are the valid values with SparkShell.

Closes #33923 from sarutak/fix-doc-about-catalogImplementation.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a5fe5d368c)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-07 11:39:53 +09:00
Jungtaek Lim e16c886b87 [SPARK-36667][SS][TEST] Close resources properly in StateStoreSuite/RocksDBStateStoreSuite
### What changes were proposed in this pull request?

This PR proposes to ensure StateStoreProvider instances are properly closed for each test in StateStoreSuite/RocksDBStateStoreSuite.

### Why are the changes needed?

While this doesn't break the test, this is a bad practice and may possibly make nasty problems in the future.

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

No.

### How was this patch tested?

Existing UTs

Closes #33916 from HeartSaVioR/SPARK-36667.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 093c2080fe)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-09-06 17:40:13 -07:00
Kent Yao aa96a374b2 [SPARK-36659][SQL] Promote spark.sql.execution.topKSortFallbackThreshold to a user-facing config
### What changes were proposed in this pull request?

Promote spark.sql.execution.topKSortFallbackThreshold to a user-facing config

### Why are the changes needed?

spark.sql.execution.topKSortFallbackThreshold now is an internal config hidden from users Integer.MAX_VALUE - 15 as its default. In many real-world cases, if the K is very big,  there would be performance issues.

It's better to leave this choice to users

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

 spark.sql.execution.topKSortFallbackThreshold is now user-facing

### How was this patch tested?

passing GA

Closes #33904 from yaooqinn/SPARK-36659.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit 7f1ad7be18)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-09-03 10:27:10 -07:00
Kousuke Saruta a3901ed384 [SPARK-36639][SQL] Fix an issue that sequence builtin function causes ArrayIndexOutOfBoundsException if the arguments are under the condition of start == stop && step < 0
### What changes were proposed in this pull request?

This PR fixes an issue that `sequence` builtin function causes `ArrayIndexOutOfBoundsException` if the arguments are under the condition of `start == stop && step < 0`.
This is an example.
```
SELECT sequence(timestamp'2021-08-31', timestamp'2021-08-31', -INTERVAL 1 month);
21/09/02 04:14:42 ERROR SparkSQLDriver: Failed in [SELECT sequence(timestamp'2021-08-31', timestamp'2021-08-31', -INTERVAL 1 month)]
java.lang.ArrayIndexOutOfBoundsException: 1
```
Actually, this example succeeded before SPARK-31980 (#28819) was merged.

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

New tests.

Closes #33895 from sarutak/fix-sequence-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
(cherry picked from commit cf3bc65e69)
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-09-03 23:25:33 +09:00
William Hyun 99f6f7f8f8 [SPARK-36657][SQL] Update comment in 'gen-sql-config-docs.py'
### What changes were proposed in this pull request?
This PR aims to update comments in `gen-sql-config-docs.py`.

### Why are the changes needed?
To make it up to date according to Spark version 3.2.0 release.

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

### How was this patch tested?
N/A.

Closes #33902 from williamhyun/fixtool.

Authored-by: William Hyun <william@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit b72fa5ef1c)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-09-02 18:51:10 -07:00
Angerszhuuuu 8b4cc90c44 [SPARK-36637][SQL] Provide proper error message when use undefined window frame
### What changes were proposed in this pull request?
Two case of using undefined window frame as below should provide proper error message

1. For case using undefined window frame with window function
```
SELECT nth_value(employee_name, 2) OVER w second_highest_salary
FROM basic_pays;
```
origin error message is
```
Window function nth_value(employee_name#x, 2, false) requires an OVER clause.
```
It's confused that in use use a window frame `w` but it's not defined.
Now the error message is
```
Window specification w is not defined in the WINDOW clause.
```

2. For case using undefined window frame with aggregation function
```
SELECT SUM(salary) OVER w sum_salary
FROM basic_pays;
```
origin error message is
```
Error in query: unresolved operator 'Aggregate [unresolvedwindowexpression(sum(salary#2), WindowSpecReference(w)) AS sum_salary#34]
+- SubqueryAlias spark_catalog.default.basic_pays
+- HiveTableRelation [`default`.`employees`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [name#0, dept#1, salary#2, age#3], Partition Cols: []]
```
In this case, when convert GlobalAggregate, should skip UnresolvedWindowExpression
Now the error message is
```
Window specification w is not defined in the WINDOW clause.
```

### Why are the changes needed?
Provide proper error message

### Does this PR introduce _any_ user-facing change?
Yes, error messages are improved as described in desc

### How was this patch tested?
Added UT

Closes #33892 from AngersZhuuuu/SPARK-36637.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 568ad6aa44)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-09-02 22:32:47 +08:00
Cary Lee 11d10fc994
[SPARK-36617][PYTHON] Fix type hints for approxQuantile to support multi-column version
### What changes were proposed in this pull request?
Update both `DataFrame.approxQuantile` and `DataFrameStatFunctions.approxQuantile` to support overloaded definitions when multiple columns are supplied.

### Why are the changes needed?
The current type hints don't support the multi-column signature, a form that was added in Spark 2.2 (see [the approxQuantile docs](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.DataFrame.approxQuantile.html).) This change was also introduced to pyspark-stubs (https://github.com/zero323/pyspark-stubs/pull/552). zero323 asked me to open a PR for the upstream change.

### Does this PR introduce _any_ user-facing change?
This change only affects type hints - it brings the `approxQuantile` type hints up to date with the actual code.

### How was this patch tested?
Ran `./dev/lint-python`.

Closes #33880 from carylee/master.

Authored-by: Cary Lee <cary@amperity.com>
Signed-off-by: zero323 <mszymkiewicz@gmail.com>
(cherry picked from commit 37f5ab07fa)
Signed-off-by: zero323 <mszymkiewicz@gmail.com>
2021-09-02 15:03:08 +02:00
Hyukjin Kwon e9f2e34261 [SPARK-36631][R] Ask users if they want to download and install SparkR in non Spark scripts
### What changes were proposed in this pull request?

This PR proposes to ask users if they want to download and install SparkR when they install SparkR from CRAN.

`SPARKR_ASK_INSTALLATION` environment variable was added in case other notebook projects are affected.

### Why are the changes needed?

This is required for CRAN. Currently SparkR is removed: https://cran.r-project.org/web/packages/SparkR/index.html.
See also https://lists.apache.org/thread.html/r02b9046273a518e347dfe85f864d23d63d3502c6c1edd33df17a3b86%40%3Cdev.spark.apache.org%3E

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

Yes, `sparkR.session(...)` will ask if users want to download and install Spark package or not if they are in the plain R shell or `Rscript`.

### How was this patch tested?

**R shell**

Valid input (`n`):

```
> sparkR.session(master="local")
Spark not found in SPARK_HOME:
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): n
```
```
Error in sparkCheckInstall(sparkHome, master, deployMode) :
  Please make sure Spark package is installed in this machine.
- If there is one, set the path in sparkHome parameter or environment variable SPARK_HOME.
- If not, you may run install.spark function to do the job.
```

Invalid input:

```
> sparkR.session(master="local")
Spark not found in SPARK_HOME:
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): abc
```
```
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n):
```

Valid input (`y`):

```
> sparkR.session(master="local")
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): y
Spark not found in the cache directory. Installation will start.
MirrorUrl not provided.
Looking for preferred site from apache website...
Preferred mirror site found: https://ftp.riken.jp/net/apache/spark
Downloading spark-3.3.0 for Hadoop 2.7 from:
- https://ftp.riken.jp/net/apache/spark/spark-3.3.0/spark-3.3.0-bin-hadoop2.7.tgz
trying URL 'https://ftp.riken.jp/net/apache/spark/spark-3.3.0/spark-3.3.0-bin-hadoop2.7.tgz'
...
```

**Rscript**

```
cat tmp.R
```
```
library(SparkR, lib.loc = c(file.path(".", "R", "lib")))
sparkR.session(master="local")
```

```
Rscript tmp.R
```

Valid input (`n`):

```
Spark not found in SPARK_HOME:
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): n
```
```
Error in sparkCheckInstall(sparkHome, master, deployMode) :
  Please make sure Spark package is installed in this machine.
- If there is one, set the path in sparkHome parameter or environment variable SPARK_HOME.
- If not, you may run install.spark function to do the job.
Calls: sparkR.session -> sparkCheckInstall
```

Invalid input:

```
Spark not found in SPARK_HOME:
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): abc
```
```
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n):
```

Valid input (`y`):

```
...
Spark not found in SPARK_HOME:
Will you download and install (or reuse if it exists) Spark package under the cache [/.../Caches/spark]? (y/n): y
Spark not found in the cache directory. Installation will start.
MirrorUrl not provided.
Looking for preferred site from apache website...
Preferred mirror site found: https://ftp.riken.jp/net/apache/spark
Downloading spark-3.3.0 for Hadoop 2.7 from:
...
```

`bin/sparkR` and `bin/spark-submit *.R` are not affected (tested).

Closes #33887 from HyukjinKwon/SPARK-36631.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit e983ba8fce)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-02 13:27:55 +09:00
Leona Yoda b81e9741cd [SPARK-36621][PYTHON][DOCS] Add Apache license headers to Pandas API on Spark documents
### What changes were proposed in this pull request?

 Apache license headers to Pandas API on Spark documents.

### Why are the changes needed?

Pandas API on Spark document sources do not have license headers, while the other docs have.

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

No

### How was this patch tested?

`make html`

Closes #33871 from yoda-mon/add-license-header.

Authored-by: Leona Yoda <yodal@oss.nttdata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 77fdf5f0e4)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-09-02 12:35:51 +09:00
Dongjoon Hyun cdf21f729a [SPARK-36629][BUILD] Upgrade aircompressor to 1.21
### What changes were proposed in this pull request?

This PR aims to upgrade `aircompressor` dependency from 1.19 to 1.21.

### Why are the changes needed?

This will bring the latest bug fix which exists in `aircompressor` 1.17 ~ 1.20.
- 1e364f7133

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

No.

### How was this patch tested?

Pass the CIs.

Closes #33883 from dongjoon-hyun/SPARK-36629.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit ff8cc4b800)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-31 22:35:53 -07:00
Gengliang Wang 1bad04d028 Preparing development version 3.2.1-SNAPSHOT 2021-08-31 17:04:14 +00:00
Gengliang Wang 03f5d23e96 Preparing Spark release v3.2.0-rc2 2021-08-31 17:04:08 +00:00
Jungtaek Lim 9a71c4ca84 [SPARK-36619][SS] Fix bugs around prefix-scan for HDFS backed state store and RocksDB state store
### What changes were proposed in this pull request?

This PR proposes to fix bugs around prefix-scan for both HDFS backed state store and RocksDB state store.

> HDFS backed state store

We did "shallow-copy" on copying prefix map, which leads the values of prefix map (mutable Set) to be "same instances" across multiple versions. This PR fixes it via creating a new mutable Set and copying elements.

> RocksDB state store

Prefix-scan iterators are only closed on RocksDB.rollback(), which is only called in RocksDBStateStore.abort().

While `RocksDBStateStore.abort()` method will be called for streaming session window (since it has two physical plans for read and write), other stateful operators which only have read-write physical plan will call either commit or abort, and don't close the iterators on committing. These unclosed iterators can be "reused" and produce incorrect outputs.

This PR ensures that resetting prefix-scan iterators is done on loading RocksDB, which was only done in rollback.

### Why are the changes needed?

Please refer the above section on explanation of bugs and treatments.

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

No.

### How was this patch tested?

Modified UT which failed without this PR and passes with this PR.

Closes #33870 from HeartSaVioR/SPARK-36619.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 60a72c938a)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-09-01 00:51:58 +08:00
yi.wu 8be53c3298 [SPARK-36614][CORE][UI] Correct executor loss reason caused by decommission in UI
### What changes were proposed in this pull request?

Post the correct executor loss reason to UI.

### Why are the changes needed?

To show the accurate loss reason.

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

Yes. Users can see the difference from the UI.

Before:
<img width="509" alt="WeChataad8d1f27d9f9aa7cf93ced4bcc820e2" src="https://user-images.githubusercontent.com/16397174/131341692-6f412607-87b8-405e-822d-0d28f07928da.png">
<img width="1138" alt="WeChat13c9f1345a096ff83d193e4e9853b165" src="https://user-images.githubusercontent.com/16397174/131341699-f2c9de09-635f-49df-8e27-2495f34276c0.png">

After:

<img width="599" alt="WeChata4313fa2dbf27bf2dbfaef5c1d4a19cf" src="https://user-images.githubusercontent.com/16397174/131341754-e3c93b5d-5252-4006-a4cc-94d76f41303b.png">
<img width="1182" alt="WeChat5559d52fd3070ae6c42fe32d56f9dc94" src="https://user-images.githubusercontent.com/16397174/131341761-e1e0644f-1e76-49c0-915a-26aad77ec272.png">

### How was this patch tested?

Manully tested.

Closes #33868 from Ngone51/fix-executor-remove-reason.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit ebe7bb6217)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-08-30 09:09:34 -07:00
gengjiaan d42536a6ee [SPARK-36574][SQL] pushDownPredicate=false should prevent push down filters to JDBC data source
### What changes were proposed in this pull request?
Spark SQL includes a data source that can read data from other databases using JDBC.
Spark also supports the case-insensitive option `pushDownPredicate`.
According to http://spark.apache.org/docs/latest/sql-data-sources-jdbc.html, If set `pushDownPredicate` to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark.
But I find it still be pushed down to JDBC data source.

### Why are the changes needed?
Fix bug `pushDownPredicate`=false failed to prevent push down filters to JDBC data source.

### Does this PR introduce _any_ user-facing change?
'No'.
The output of query will not change.

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

Closes #33822 from beliefer/SPARK-36574.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit fcc91cfec4)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-30 19:09:45 +08:00
Sean Owen b76471c5df [SPARK-36603][CORE] Use WeakReference not SoftReference in LevelDB
### What changes were proposed in this pull request?

Use WeakReference not SoftReference in LevelDB

### Why are the changes needed?

(See discussion at https://github.com/apache/spark/pull/28769#issuecomment-906722390 )

"The soft reference to iterator introduced in this pr unfortunately ended up causing iterators to not be closed when they go out of scope (which would have happened earlier in the finalize)

This is because java is more conservative in cleaning up SoftReference's.
The net result was we ended up having 50k files for SHS while typically they get compacted away to 200 odd files.

Changing from SoftReference to WeakReference should make it much more aggresive in cleanup and prevent the issue - which we observed in a 3.1 SHS"

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

No

### How was this patch tested?

Existing tests

Closes #33859 from srowen/SPARK-36603.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 89e907f76c)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-29 09:30:20 -07:00
Hyukjin Kwon e3edb65bf0 [MINOR][DOCS] Add Apache license header to GitHub Actions workflow files
### What changes were proposed in this pull request?

Some of GitHub Actions workflow files do not have the Apache license header. This PR adds them.

### Why are the changes needed?

To comply Apache license.

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

No.

### How was this patch tested?

N/A

Closes #33862 from HyukjinKwon/minor-lisence.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 22c492a6b8)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-28 20:30:25 -07:00
Gengliang Wang 3719d87668 [SPARK-36606][DOCS][TESTS] Enhance the docs and tests of try_add/try_divide
### What changes were proposed in this pull request?

The `try_add` function allows the following inputs:
- number, number
- date, number
- date, interval
- timestamp, interval
- interval, interval

And, the `try_divide` function allows the following inputs:

- number, number
- interval, number

However, in the current code, there are only examples and tests about the (number, number) inputs. We should enhance the docs to let users know that the functions can be used for datetime and interval operations too.

### Why are the changes needed?

Improve documentation and tests.

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

No

### How was this patch tested?

New UT
Also build docs for preview:
![image](https://user-images.githubusercontent.com/1097932/131212897-8aea14c8-a882-4e12-94e2-f56bde7c0367.png)

Closes #33861 from gengliangwang/enhanceTryDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 8a52ad9f82)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-29 10:30:13 +09:00
Kousuke Saruta 93f2b00501 [SPARK-36509][CORE] Fix the issue that executors are never re-scheduled if the worker stops with standalone cluster
### What changes were proposed in this pull request?

This PR fixes an issue that executors are never re-scheduled if the worker which the executors run on stops.
As a result, the application stucks.
You can easily reproduce this issue by the following procedures.

```
# Run master
$ sbin/start-master.sh

# Run worker 1
$ SPARK_LOG_DIR=/tmp/worker1 SPARK_PID_DIR=/tmp/worker1/ sbin/start-worker.sh -c 1 -h localhost -d /tmp/worker1 --webui-port 8081 spark://<hostname>:7077

# Run worker 2
$ SPARK_LOG_DIR=/tmp/worker2 SPARK_PID_DIR=/tmp/worker2/ sbin/start-worker.sh -c 1 -h localhost -d /tmp/worker2 --webui-port 8082 spark://<hostname>:7077

# Run Spark Shell
$ bin/spark-shell --master spark://<hostname>:7077 --executor-cores 1 --total-executor-cores 1

# Check which worker the executor runs on and then kill the worker.
$ kill <worker pid>
```

With the procedure above, we will expect that the executor is re-scheduled on the other worker but it won't.

The reason seems that `Master.schedule` cannot be called after the worker is marked as `WorkerState.DEAD`.
So, the solution this PR proposes is to call `Master.schedule` whenever `Master.removeWorker` is called.

This PR also fixes an issue that `ExecutorRunner` can send `ExecutorStateChanged` message without changing its state.
This issue causes assertion error.
```
2021-08-13 14:05:37,991 [dispatcher-event-loop-9] ERROR: Ignoring errorjava.lang.AssertionError: assertion failed: executor 0 state transfer from RUNNING to RUNNING is illegal
```

### Why are the changes needed?

It's a critical bug.

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

No.

### How was this patch tested?

Manually tested with the procedure shown above and confirmed the executor is re-scheduled.

Closes #33818 from sarutak/fix-scheduling-stuck.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
(cherry picked from commit ea8c31e5ea)
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-08-28 18:02:11 +09:00
Gengliang Wang 9b28c2b09e [SPARK-36597][DOCS][3.2] Fix issues in SQL function docs
### What changes were proposed in this pull request?

* the functions make_dt_interval and make_ym_interval should make it clear that some of the fields are optional
* remove the `|` symbol from the doc of `bit_get` https://dist.apache.org/repos/dist/dev/spark/v3.2.0-rc1-docs/_site/api/sql/index.html#bit_get
* Address one missing comment in https://github.com/apache/spark/pull/33824#discussion_r695405699

### Why are the changes needed?

Improve the documentation.

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

No

### How was this patch tested?

Build doc and preview:
![image](https://user-images.githubusercontent.com/1097932/130996918-8c1fff88-ef5a-434b-8445-df7140bad3ba.png)
![image](https://user-images.githubusercontent.com/1097932/130996954-0ced28e7-fb90-4fcc-857e-6ccc31dc3c09.png)

![image](https://user-images.githubusercontent.com/1097932/130955106-5ae32dfc-6e89-4e28-bb8a-6c1b5213051c.png)

![image](https://user-images.githubusercontent.com/1097932/130922351-2f0f262d-5624-4d08-ba83-dfa3ed0b646b.png)

Closes #33857 from gengliangwang/SPARK-36597-3.2.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-27 13:00:12 -07:00
itholic 396b76466b [SPARK-36388][SPARK-36386][PYTHON][FOLLOWUP] Fix DataFrame groupby-rolling and groupby-expanding to follow pandas 1.3
This PR is followup for https://github.com/apache/spark/pull/33646 to add missing tests.

Some tests are missing

No

Unittest

Closes #33776 from itholic/SPARK-36388-followup.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit c91ae544fd)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-27 20:46:54 +09:00