Commit graph

30981 commits

Author SHA1 Message Date
Gengliang Wang 5463caac0d Revert "[SPARK-34415][ML] Randomization in hyperparameter optimization"
### What changes were proposed in this pull request?

Revert 397b843890 and 5a48eb8d00

### Why are the changes needed?

As discussed in https://github.com/apache/spark/pull/33800#issuecomment-904140869, there is correctness issue in the current implementation. Let's revert the code changes from branch 3.2 and fix it on master branch later

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

No

### How was this patch tested?

Ci tests

Closes #33819 from gengliangwang/revert-SPARK-34415.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit de932f51ce)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-24 13:39:29 -07:00
yi.wu 36df86c0d0 [SPARK-36564][CORE] Fix NullPointerException in LiveRDDDistribution.toApi
### What changes were proposed in this pull request?

This PR fixes `NullPointerException` in `LiveRDDDistribution.toApi`.

### Why are the changes needed?

Looking at the stack trace, the NPE is caused by the null `exec.hostPort`. I can't get the complete log to take a close look but only guess that it might be due to the event `SparkListenerBlockManagerAdded` is dropped or out of order.

```
21/08/23 12:26:29 ERROR AsyncEventQueue: Listener AppStatusListener threw an exception
java.lang.NullPointerException
	at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:192)
	at com.google.common.collect.MapMakerInternalMap.putIfAbsent(MapMakerInternalMap.java:3507)
	at com.google.common.collect.Interners$WeakInterner.intern(Interners.java:85)
	at org.apache.spark.status.LiveEntityHelpers$.weakIntern(LiveEntity.scala:696)
	at org.apache.spark.status.LiveRDDDistribution.toApi(LiveEntity.scala:563)
	at org.apache.spark.status.LiveRDD.$anonfun$doUpdate$4(LiveEntity.scala:629)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.mutable.HashMap$$anon$2.$anonfun$foreach$3(HashMap.scala:158)
	at scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
	at scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
	at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
	at scala.collection.mutable.HashMap$$anon$2.foreach(HashMap.scala:158)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at scala.collection.AbstractTraversable.map(Traversable.scala:108)
	at org.apache.spark.status.LiveRDD.doUpdate(LiveEntity.scala:629)
	at org.apache.spark.status.LiveEntity.write(LiveEntity.scala:51)
	at org.apache.spark.status.AppStatusListener.update(AppStatusListener.scala:1206)
	at org.apache.spark.status.AppStatusListener.maybeUpdate(AppStatusListener.scala:1212)
	at org.apache.spark.status.AppStatusListener.$anonfun$onExecutorMetricsUpdate$6(AppStatusListener.scala:956)
	...
```

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

Yes, users will see the expected RDD info in UI instead of the NPE error.

 ### How was this patch tested?

Pass existing tests.

Closes #33812 from Ngone51/fix-hostport-npe.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit d6c453aaea)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-24 13:33:51 -07:00
Gengliang Wang a313082d67 [SPARK-35535][SQL][FOLLOWUP] Move LocalScan to Catalyst package
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/32678. It moves `LocalScan` from SQL core package to Catalyst package.

### Why are the changes needed?

There are two packages for `org.apache.spark.sql.connector`
SQL Core: https://github.com/apache/spark/tree/master/sql/core/src/main/java/org/apache/spark/sql/connector
Catalyst: https://github.com/apache/spark/tree/master/sql/catalyst/src/main/java/org/apache/spark/sql/connector

As `LocalScan` doesn't depend on the classes of SQL Core, we should move it to catalyst.
### Does this PR introduce _any_ user-facing change?

No, the trait is not released yet.

### How was this patch tested?

Existing UT.

Closes #33826 from gengliangwang/moveLocalScan.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 5b4c216478)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-24 13:24:06 -07:00
Huaxin Gao e48de7884d [SPARK-34952][SQL][FOLLOWUP] Move aggregates to a separate package
### What changes were proposed in this pull request?
Add `aggregate` package under `sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions` and move all the aggregates (e.g. `Count`, `Max`, `Min`, etc.) there.

### Why are the changes needed?
Right now these aggregates are under `sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions`. It looks OK now, but we plan to add a new `filter` package under `expressions` for all the DSV2 filters. It will look strange that filters have their own package, but aggregates don't.

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

### How was this patch tested?
Existing tests

Closes #33815 from huaxingao/agg_package.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit cd2342691d)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-08-23 15:31:35 -07:00
Xinrong Meng 56c211bd6a [SPARK-36470][PYTHON] Implement CategoricalIndex.map and DatetimeIndex.map
Implement `CategoricalIndex.map` and `DatetimeIndex.map`

`MultiIndex.map` cannot be implemented in the same way as the `map` of other indexes. It should be taken care of separately if necessary.

Mapping values using input correspondence is a common operation that is supported in pandas. We shall support that as well.

Yes. `CategoricalIndex.map` and `DatetimeIndex.map` can be used now.

- CategoricalIndex.map

```py
>>> idx = ps.CategoricalIndex(['a', 'b', 'c'])
>>> idx
CategoricalIndex(['a', 'b', 'c'], categories=['a', 'b', 'c'], ordered=False, dtype='category')

>>> idx.map(lambda x: x.upper())
CategoricalIndex(['A', 'B', 'C'],  categories=['A', 'B', 'C'], ordered=False, dtype='category')

>>> pser = pd.Series([1, 2, 3], index=pd.CategoricalIndex(['a', 'b', 'c'], ordered=True))
>>> idx.map(pser)
CategoricalIndex([1, 2, 3], categories=[1, 2, 3], ordered=True, dtype='category')

>>> idx.map({'a': 'first', 'b': 'second', 'c': 'third'})
CategoricalIndex(['first', 'second', 'third'], categories=['first', 'second', 'third'], ordered=False, dtype='category')
```

- DatetimeIndex.map

```py
>>> pidx = pd.date_range(start="2020-08-08", end="2020-08-10")
>>> psidx = ps.from_pandas(pidx)

>>> mapper_dict = {
...   datetime.datetime(2020, 8, 8): datetime.datetime(2021, 8, 8),
...   datetime.datetime(2020, 8, 9): datetime.datetime(2021, 8, 9),
... }
>>> psidx.map(mapper_dict)
DatetimeIndex(['2021-08-08', '2021-08-09', 'NaT'], dtype='datetime64[ns]', freq=None)

>>> mapper_pser = pd.Series([1, 2, 3], index=pidx)
>>> psidx.map(mapper_pser)
Int64Index([1, 2, 3], dtype='int64')
>>> psidx
DatetimeIndex(['2020-08-08', '2020-08-09', '2020-08-10'], dtype='datetime64[ns]', freq=None)

>>> psidx.map(lambda x: x.strftime("%B %d, %Y, %r"))
Index(['August 08, 2020, 12:00:00 AM', 'August 09, 2020, 12:00:00 AM',
       'August 10, 2020, 12:00:00 AM'],
      dtype='object')
```

Unit tests.

Closes #33756 from xinrong-databricks/other_indexes_map.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 0b6af464dc)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-23 10:11:21 +09:00
Gengliang Wang eea7d0037e [SPARK-36557][DOCS] Update the MAVEN_OPTS in Spark build docs
### What changes were proposed in this pull request?

As Jacek Laskowski pointed out in the dev list, there is StackOverflowError if compiling Spark with the current MAVEN_OPTS in Spark documentation.
We should update it with `-Xss64m` to avoid it.

### Why are the changes needed?

Correct the documentation

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

No

### How was this patch tested?

Manual test. The MAVEN_OPTS is consistent with our github action build.

Closes #33804 from gengliangwang/updateBuildDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 3da0e9500f)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-23 09:46:41 +09:00
Venkata krishnan Sowrirajan 0f2e318894 [SPARK-36374][FOLLOW-UP] Change config key spark.shuffle.server.mergedShuffleFileManagerImpl to spark.shuffle.push.server.mergedShuffleFileManagerImpl
### What changes were proposed in this pull request?

Minor changes to change the config key name from `spark.shuffle.server.mergedShuffleFileManagerImpl` to `spark.shuffle.push.server.mergedShuffleFileManagerImpl`. This is missed out in https://github.com/apache/spark/pull/33615.

### Why are the changes needed?

To keep the config names consistent

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

Yes, this is a change in the config key name. But the new config name changes are yet to be released. Technically there is no user facing change because of this change.

### How was this patch tested?

Existing tests.

Closes #33799 from venkata91/SPARK-36374-follow-up.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 7b2842e986)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-22 01:29:36 -05:00
Liang-Chi Hsieh 212a21ee4f [MINOR][SS][DOCS] Update doc for streaming deduplication
### What changes were proposed in this pull request?

This patch fixes an error about streaming dedupliaction is Structured Streaming, and also updates an item about unsupported operation.

### Why are the changes needed?

Update the user document.

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

No. It's a doc only change.

### How was this patch tested?

Doc only change.

Closes #33801 from viirya/minor-ss-deduplication.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 5876e04de2)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-08-21 18:20:27 -07:00
Angerszhuuuu 45c4b751f3 [SPARK-36549][SQL] Add taskStatus supports multiple value to monitoring doc
### What changes were proposed in this pull request?
In Stage related restful API, we support `taskStatus` parameter as a list
```
 QueryParam("taskStatus") taskStatus: JList[TaskStatus]
```
In restful we should write like
```
taskStatus=SUCCESS&taskStatus=FAILED
```

It's usefule but not show in the doc, and many user don't know how to write the list parameters.
So add this feature to monitoring doc too.

### Why are the changes needed?
Make doc clear

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

### How was this patch tested?
With restful request
```
http://localhost:4040/api/v1/applications/local-1629432414554/stages/0?details=true&taskStatus=FAILED
```
Resultful request result tasks
```
tasks" : {
    "0" : {
      "taskId" : 0,
      "index" : 0,
      "attempt" : 0,
      "launchTime" : "2021-08-20T04:06:55.515GMT",
      "duration" : 273,
      "executorId" : "driver",
      "host" : "host",
      "status" : "FAILED",
      "taskLocality" : "PROCESS_LOCAL",
      "speculative" : false,
      "accumulatorUpdates" : [ ],
      "errorMessage" : "java.lang.RuntimeException\n\tat org.apache.spark.ui.UISuite.$anonfun$new$8(UISuite.scala:95)\n\tat scala.runtime.java8.JFunction1$mcVI$sp.apply(JFunction1$mcVI$sp.java:23)\n\tat scala.collection.Iterator.foreach(Iterator.scala:943)\n\tat scala.collection.Iterator.foreach$(Iterator.scala:943)\n\tat org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28)\n\tat org.apache.spark.rdd.RDD.$anonfun$foreach$2(RDD.scala:1003)\n\tat org.apache.spark.rdd.RDD.$anonfun$foreach$2$adapted(RDD.scala:1003)\n\tat org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2254)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:136)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:507)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1468)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:510)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
      "taskMetrics" : {
        "executorDeserializeTime" : 0,
        "executorDeserializeCpuTime" : 0,
        "executorRunTime" : 206,
        "executorCpuTime" : 0,
        "resultSize" : 0,
        "jvmGcTime" : 0,
        "resultSerializationTime" : 0,
        "memoryBytesSpilled" : 0,
        "diskBytesSpilled" : 0,
        "peakExecutionMemory" : 0,
        "inputMetrics" : {
          "bytesRead" : 0,
          "recordsRead" : 0
        },
        "outputMetrics" : {
          "bytesWritten" : 0,
          "recordsWritten" : 0
        },
        "shuffleReadMetrics" : {
          "remoteBlocksFetched" : 0,
          "localBlocksFetched" : 0,
          "fetchWaitTime" : 0,
          "remoteBytesRead" : 0,
          "remoteBytesReadToDisk" : 0,
          "localBytesRead" : 0,
          "recordsRead" : 0
        },
        "shuffleWriteMetrics" : {
          "bytesWritten" : 0,
          "writeTime" : 0,
          "recordsWritten" : 0
        }
      },
      "executorLogs" : { },
      "schedulerDelay" : 67,
      "gettingResultTime" : 0
    }
  },
```

With restful request
```
http://localhost:4040/api/v1/applications/local-1629432414554/stages/0?details=true&taskStatus=FAILED&taskStatus=SUCCESS
```
Restful result tasks
```
"tasks" : {
    "1" : {
      "taskId" : 1,
      "index" : 1,
      "attempt" : 0,
      "launchTime" : "2021-08-20T04:06:55.786GMT",
      "duration" : 16,
      "executorId" : "driver",
      "host" : "host",
      "status" : "SUCCESS",
      "taskLocality" : "PROCESS_LOCAL",
      "speculative" : false,
      "accumulatorUpdates" : [ ],
      "taskMetrics" : {
        "executorDeserializeTime" : 2,
        "executorDeserializeCpuTime" : 2638000,
        "executorRunTime" : 2,
        "executorCpuTime" : 1993000,
        "resultSize" : 837,
        "jvmGcTime" : 0,
        "resultSerializationTime" : 0,
        "memoryBytesSpilled" : 0,
        "diskBytesSpilled" : 0,
        "peakExecutionMemory" : 0,
        "inputMetrics" : {
          "bytesRead" : 0,
          "recordsRead" : 0
        },
        "outputMetrics" : {
          "bytesWritten" : 0,
          "recordsWritten" : 0
        },
        "shuffleReadMetrics" : {
          "remoteBlocksFetched" : 0,
          "localBlocksFetched" : 0,
          "fetchWaitTime" : 0,
          "remoteBytesRead" : 0,
          "remoteBytesReadToDisk" : 0,
          "localBytesRead" : 0,
          "recordsRead" : 0
        },
        "shuffleWriteMetrics" : {
          "bytesWritten" : 0,
          "writeTime" : 0,
          "recordsWritten" : 0
        }
      },
      "executorLogs" : { },
      "schedulerDelay" : 12,
      "gettingResultTime" : 0
    },
    "0" : {
      "taskId" : 0,
      "index" : 0,
      "attempt" : 0,
      "launchTime" : "2021-08-20T04:06:55.515GMT",
      "duration" : 273,
      "executorId" : "driver",
      "host" : "host",
      "status" : "FAILED",
      "taskLocality" : "PROCESS_LOCAL",
      "speculative" : false,
      "accumulatorUpdates" : [ ],
      "errorMessage" : "java.lang.RuntimeException\n\tat org.apache.spark.ui.UISuite.$anonfun$new$8(UISuite.scala:95)\n\tat scala.runtime.java8.JFunction1$mcVI$sp.apply(JFunction1$mcVI$sp.java:23)\n\tat scala.collection.Iterator.foreach(Iterator.scala:943)\n\tat scala.collection.Iterator.foreach$(Iterator.scala:943)\n\tat org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28)\n\tat org.apache.spark.rdd.RDD.$anonfun$foreach$2(RDD.scala:1003)\n\tat org.apache.spark.rdd.RDD.$anonfun$foreach$2$adapted(RDD.scala:1003)\n\tat org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2254)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:136)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:507)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1468)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:510)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n",
      "taskMetrics" : {
        "executorDeserializeTime" : 0,
        "executorDeserializeCpuTime" : 0,
        "executorRunTime" : 206,
        "executorCpuTime" : 0,
        "resultSize" : 0,
        "jvmGcTime" : 0,
        "resultSerializationTime" : 0,
        "memoryBytesSpilled" : 0,
        "diskBytesSpilled" : 0,
        "peakExecutionMemory" : 0,
        "inputMetrics" : {
          "bytesRead" : 0,
          "recordsRead" : 0
        },
        "outputMetrics" : {
          "bytesWritten" : 0,
          "recordsWritten" : 0
        },
        "shuffleReadMetrics" : {
          "remoteBlocksFetched" : 0,
          "localBlocksFetched" : 0,
          "fetchWaitTime" : 0,
          "remoteBytesRead" : 0,
          "remoteBytesReadToDisk" : 0,
          "localBytesRead" : 0,
          "recordsRead" : 0
        },
        "shuffleWriteMetrics" : {
          "bytesWritten" : 0,
          "writeTime" : 0,
          "recordsWritten" : 0
        }
      },
      "executorLogs" : { },
      "schedulerDelay" : 67,
      "gettingResultTime" : 0
    }
  },
```

Closes #33793 from AngersZhuuuu/SPARK-36549.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 5740d5641d)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-22 09:45:34 +09:00
Kent Yao bdd3b49026 [SPARK-36552][SQL] Fix different behavior for writing char/varchar to hive and datasource table
### What changes were proposed in this pull request?

For the hive table, the actual write path and the schema handling are inconsistent when `spark.sql.legacy.charVarcharAsString` is true.

This causes problems like SPARK-36552 described.

In this PR we respect `spark.sql.legacy.charVarcharAsString` when generates hive table schema from spark data types.

### Why are the changes needed?

bugfix

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

yes, when `spark.sql.legacy.charVarcharAsString` is true, hive table with char/varchar will respect string behavior.

### How was this patch tested?

newly added test

Closes #33798 from yaooqinn/SPARK-36552.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit f918c123a0)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-22 09:38:54 +09:00
ulysses-you e0d2d8f1a6 [SPARK-35083][CORE][FOLLLOWUP] Improve docs and migration guide
### What changes were proposed in this pull request?

* improve docs in `docs/job-scheduling.md`
* add migration guide docs in `docs/core-migration-guide.md`

### Why are the changes needed?

Help user to migrate.

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

yes

### How was this patch tested?

Pass CI

Closes #33794 from ulysses-you/SPARK-35083-f.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
(cherry picked from commit 90cbf9ca3e)
Signed-off-by: Kent Yao <yao@apache.org>
2021-08-20 21:33:06 +08:00
sweisdb 243bfafd5c Updates AuthEngine to pass the correct SecretKeySpec format
AuthEngineSuite was passing on some platforms (MacOS), but failing on others (Linux) with an InvalidKeyException stemming from this line. We should explicitly pass AES as the key format.

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

Changes the AuthEngine SecretKeySpec from "RAW" to "AES".

### Why are the changes needed?

Unit tests were failing on some platforms with InvalidKeyExceptions when this key was used to instantiate a Cipher.

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

No.

### How was this patch tested?

Unit tests on a MacOS and Linux platform.

Closes #33790 from sweisdb/patch-1.

Authored-by: sweisdb <60895808+sweisdb@users.noreply.github.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
(cherry picked from commit c441c7e365)
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-08-20 08:31:54 -05:00
Gengliang Wang 69be513c5e Preparing development version 3.2.1-SNAPSHOT 2021-08-20 12:40:47 +00:00
Gengliang Wang 6bb3523d8e Preparing Spark release v3.2.0-rc1 2021-08-20 12:40:40 +00:00
Gengliang Wang fafdc1482b Revert "Preparing Spark release v3.2.0-rc1"
This reverts commit 8e58fafb05.
2021-08-20 20:07:02 +08:00
Gengliang Wang c829ed53ff Revert "Preparing development version 3.2.1-SNAPSHOT"
This reverts commit 4f1d21571d.
2021-08-20 20:07:01 +08:00
Gengliang Wang f47a519721 [SPARK-36551][BUILD] Add sphinx-plotly-directive in Spark release Dockerfile
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/32726, Python doc build requires `sphinx-plotly-directive`.
This PR is to install it from `spark-rm/Dockerfile` to make sure `do-release-docker.sh` can run successfully.
Also, this PR mentions it in the README of docs.

### Why are the changes needed?

Fix release script and update README of docs

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

No
### How was this patch tested?

Manual test locally.

Closes #33797 from gengliangwang/fixReleaseDocker.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 42eebb84f5)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-20 20:02:44 +08:00
Gengliang Wang 6357b22ba8 [SPARK-36547][BUILD] Downgrade scala-maven-plugin to 4.3.0
### What changes were proposed in this pull request?

When preparing Spark 3.2.0 RC1, I hit the same issue of https://github.com/apache/spark/pull/31031.
```
[INFO] Compiling 21 Scala sources and 3 Java sources to /opt/spark-rm/output/spark-3.1.0-bin-hadoop2.7/resource-managers/yarn/target/scala-2.12/test-classes ...
[ERROR] ## Exception when compiling 24 sources to /opt/spark-rm/output/spark-3.1.0-bin-hadoop2.7/resource-managers/yarn/target/scala-2.12/test-classes
java.lang.SecurityException: class "javax.servlet.SessionCookieConfig"'s signer information does not match signer information of other classes in the same package
java.lang.ClassLoader.checkCerts(ClassLoader.java:891)
java.lang.ClassLoader.preDefineClass(ClassLoader.java:661)
```
This PR is to apply the same fix again by downgrading scala-maven-plugin to 4.3.0

### Why are the changes needed?

To unblock the release process.

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

No
### How was this patch tested?

Build test

Closes #33791 from gengliangwang/downgrade.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit f0775d215e)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-20 10:45:35 +08:00
Yuanjian Li 36c24a03bd [SPARK-35312][SS][FOLLOW-UP] More documents and checking logic for the new options
### What changes were proposed in this pull request?
Add more documents and checking logic for the new options `minOffsetPerTrigger` and `maxTriggerDelay`.

### Why are the changes needed?
Have a clear description of the behavior introduced in SPARK-35312

### Does this PR introduce _any_ user-facing change?
Yes.
If the user set minOffsetsPerTrigger > maxOffsetsPerTrigger, the new code will throw an AnalysisException. The original behavior is to ignore the maxOffsetsPerTrigger silenctly.

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

Closes #33792 from xuanyuanking/SPARK-35312-follow.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit a0b24019ed)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-08-20 10:41:54 +09:00
Gengliang Wang 4f1d21571d Preparing development version 3.2.1-SNAPSHOT 2021-08-19 14:08:32 +00:00
Gengliang Wang 8e58fafb05 Preparing Spark release v3.2.0-rc1 2021-08-19 14:08:26 +00:00
gengjiaan 7041c0f9f0 [SPARK-36428][TESTS][FOLLOWUP] Revert mistake change to DateExpressionsSuite
### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/33775 commits the debug code mistakely.
This PR revert the test path.

### Why are the changes needed?
Revoke debug code.

### Does this PR introduce _any_ user-facing change?
 'No'.
Just adjust test.

### How was this patch tested?
Revert non-ansi test path.

Closes #33787 from beliefer/SPARK-36428-followup2.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 462aa7cd3c)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-19 21:33:39 +08:00
Gengliang Wang fb56627f21 Revert "[SPARK-35083][FOLLOW-UP][CORE] Add migration guide for the re…
…mote scheduler pool files support"

This reverts commit e3902d1975. The feature is improvement instead of behavior change.

Closes #33789 from gengliangwang/revertDoc.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit b36b1c7e8a)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-19 21:30:19 +08:00
Yuming Wang 5b971650bd [SPARK-36444][SQL] Remove OptimizeSubqueries from batch of PartitionPruning
### What changes were proposed in this pull request?

Remove `OptimizeSubqueries` from batch of `PartitionPruning` to make DPP support more cases. For example:
```sql
SELECT date_id, product_id FROM fact_sk f
JOIN (select store_id + 3 as new_store_id from dim_store where country = 'US') s
ON f.store_id = s.new_store_id
```

Before this PR:
```
== Physical Plan ==
*(2) Project [date_id#3998, product_id#3999]
+- *(2) BroadcastHashJoin [store_id#4001], [new_store_id#3997], Inner, BuildRight, false
   :- *(2) ColumnarToRow
   :  +- FileScan parquet default.fact_sk[date_id#3998,product_id#3999,store_id#4001] Batched: true, DataFilters: [], Format: Parquet, PartitionFilters: [isnotnull(store_id#4001), dynamicpruningexpression(true)], PushedFilters: [], ReadSchema: struct<date_id:int,product_id:int>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#274]
      +- *(1) Project [(store_id#4002 + 3) AS new_store_id#3997]
         +- *(1) Filter ((isnotnull(country#4004) AND (country#4004 = US)) AND isnotnull((store_id#4002 + 3)))
            +- *(1) ColumnarToRow
               +- FileScan parquet default.dim_store[store_id#4002,country#4004] Batched: true, DataFilters: [isnotnull(country#4004), (country#4004 = US), isnotnull((store_id#4002 + 3))], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(country), EqualTo(country,US)], ReadSchema: struct<store_id:int,country:string>
```

After this PR:
```
== Physical Plan ==
*(2) Project [date_id#3998, product_id#3999]
+- *(2) BroadcastHashJoin [store_id#4001], [new_store_id#3997], Inner, BuildRight, false
   :- *(2) ColumnarToRow
   :  +- FileScan parquet default.fact_sk[date_id#3998,product_id#3999,store_id#4001] Batched: true, DataFilters: [], Format: Parquet, PartitionFilters: [isnotnull(store_id#4001), dynamicpruningexpression(store_id#4001 IN dynamicpruning#4007)], PushedFilters: [], ReadSchema: struct<date_id:int,product_id:int>
   :        +- SubqueryBroadcast dynamicpruning#4007, 0, [new_store_id#3997], [id=#263]
   :           +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#262]
   :              +- *(1) Project [(store_id#4002 + 3) AS new_store_id#3997]
   :                 +- *(1) Filter ((isnotnull(country#4004) AND (country#4004 = US)) AND isnotnull((store_id#4002 + 3)))
   :                    +- *(1) ColumnarToRow
   :                       +- FileScan parquet default.dim_store[store_id#4002,country#4004] Batched: true, DataFilters: [isnotnull(country#4004), (country#4004 = US), isnotnull((store_id#4002 + 3))], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(country), EqualTo(country,US)], ReadSchema: struct<store_id:int,country:string>
   +- ReusedExchange [new_store_id#3997], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#262]
```
This is because `OptimizeSubqueries` will infer more filters, so we cannot reuse broadcasts. The following is the plan if disable `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly`:
```
== Physical Plan ==
*(2) Project [date_id#3998, product_id#3999]
+- *(2) BroadcastHashJoin [store_id#4001], [new_store_id#3997], Inner, BuildRight, false
   :- *(2) ColumnarToRow
   :  +- FileScan parquet default.fact_sk[date_id#3998,product_id#3999,store_id#4001] Batched: true, DataFilters: [], Format: Parquet, PartitionFilters: [isnotnull(store_id#4001), dynamicpruningexpression(store_id#4001 IN subquery#4009)], PushedFilters: [], ReadSchema: struct<date_id:int,product_id:int>
   :        +- Subquery subquery#4009, [id=#284]
   :           +- *(2) HashAggregate(keys=[new_store_id#3997#4008], functions=[])
   :              +- Exchange hashpartitioning(new_store_id#3997#4008, 5), ENSURE_REQUIREMENTS, [id=#280]
   :                 +- *(1) HashAggregate(keys=[new_store_id#3997 AS new_store_id#3997#4008], functions=[])
   :                    +- *(1) Project [(store_id#4002 + 3) AS new_store_id#3997]
   :                       +- *(1) Filter (((isnotnull(store_id#4002) AND isnotnull(country#4004)) AND (country#4004 = US)) AND isnotnull((store_id#4002 + 3)))
   :                          +- *(1) ColumnarToRow
   :                             +- FileScan parquet default.dim_store[store_id#4002,country#4004] Batched: true, DataFilters: [isnotnull(store_id#4002), isnotnull(country#4004), (country#4004 = US), isnotnull((store_id#4002..., Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(store_id), IsNotNull(country), EqualTo(country,US)], ReadSchema: struct<store_id:int,country:string>
   +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#305]
      +- *(1) Project [(store_id#4002 + 3) AS new_store_id#3997]
         +- *(1) Filter ((isnotnull(country#4004) AND (country#4004 = US)) AND isnotnull((store_id#4002 + 3)))
            +- *(1) ColumnarToRow
               +- FileScan parquet default.dim_store[store_id#4002,country#4004] Batched: true, DataFilters: [isnotnull(country#4004), (country#4004 = US), isnotnull((store_id#4002 + 3))], Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(country), EqualTo(country,US)], ReadSchema: struct<store_id:int,country:string>
```

### Why are the changes needed?

Improve DPP to support more cases.

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

No.

### How was this patch tested?

Unit test and benchmark test:
SQL | Before this PR(Seconds) | After this PR(Seconds)
-- | -- | --
TPC-DS q58 | 40 | 20
TPC-DS q83 | 18 | 14

Closes #33664 from wangyum/SPARK-36444.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
(cherry picked from commit 2310b99e14)
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-08-19 16:45:22 +08:00
yi.wu 9544c24560 [SPARK-35083][FOLLOW-UP][CORE] Add migration guide for the remote scheduler pool files support
### What changes were proposed in this pull request?

Add remote scheduler pool files support to the migration guide.

### Why are the changes needed?

To highlight this useful support.

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

No.

### How was this patch tested?

Pass exiting tests.

Closes #33785 from Ngone51/SPARK-35083-follow-up.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit e3902d1975)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-19 16:29:19 +08:00
Shixiong Zhu 54cca7f82e [SPARK-36519][SS] Store RocksDB format version in the checkpoint for streaming queries
### What changes were proposed in this pull request?

RocksDB provides backward compatibility but it doesn't always provide forward compatibility. It's better to store the RocksDB format version in the checkpoint so that it would give us more information to provide the rollback guarantee when we upgrade the RocksDB version that may introduce incompatible change in a new Spark version.

A typical case is when a user upgrades their query to a new Spark version, and this new Spark version has a new RocksDB version which may use a new format. But the user hits some bug and decide to rollback. But in the old Spark version, the old RocksDB version cannot read the new format.

In order to handle this case, we will write the RocksDB format version to the checkpoint. When restarting from a checkpoint, we will force RocksDB to use the format version stored in the checkpoint. This will ensure the user can rollback their Spark version if needed.

We also provide a config `spark.sql.streaming.stateStore.rocksdb.formatVersion` for users who don't need to rollback their Spark versions to overwrite the format version specified in the checkpoint.

### Why are the changes needed?

Provide the Spark version rollback guarantee for streaming queries when a new RocksDB introduces an incompatible format change.

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

No. RocksDB state store is a new feature in Spark 3.2, which has not yet released.

### How was this patch tested?

The new unit tests.

Closes #33749 from zsxwing/SPARK-36519.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit ea4919801a)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-08-19 00:23:52 -07:00
Wenchen Fan 8f3b4c4b7d [SPARK-33687][SQL][DOC][FOLLOWUP] Merge the doc pages of ANALYZE TABLE and ANALYZE TABLES
### What changes were proposed in this pull request?

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

ANALYZE TABLE and TABLES are essentially the same command, it's weird to put them in 2 different doc pages. This PR proposes to merge them into one doc page.

### Why are the changes needed?

simplify the doc

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

no

### How was this patch tested?

N/A

Closes #33781 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 07d173a8b0)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-19 11:04:20 +08:00
gengjiaan 3d69d0d003 [SPARK-36428][SQL][FOLLOWUP] Simplify the implementation of make_timestamp
### What changes were proposed in this pull request?
The implement of https://github.com/apache/spark/pull/33665 make `make_timestamp` could accepts integer type as the seconds parameter.
This PR let `make_timestamp` accepts `decimal(16,6)` type as the seconds parameter and cast integer to `decimal(16,6)` is safe, so we can simplify the code.

### Why are the changes needed?
Simplify `make_timestamp`.

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

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

Closes #33775 from beliefer/SPARK-36428-followup.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 707eefa3c7)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-18 22:57:27 +08:00
yi.wu 181d33e16e [SPARK-36532][CORE] Fix deadlock in CoarseGrainedExecutorBackend.onDisconnected to avoid executor shutdown hang
### What changes were proposed in this pull request?

Instead of exiting the executor within the RpcEnv's thread, exit the executor in a separate thread.

### Why are the changes needed?

The current exit way in `onDisconnected` can cause the deadlock, which has the exact same root cause with https://github.com/apache/spark/pull/12012:

* `onDisconnected` -> `System.exit` are called in sequence in the thread of `MessageLoop.threadpool`
* `System.exit` triggers shutdown hooks and `executor.stop` is one of the hooks.
* `executor.stop` stops the `Dispatcher`, which waits for the `MessageLoop.threadpool`  to shutdown further.
* Thus, the thread which runs `System.exit` waits for hooks to be done, but the `MessageLoop.threadpool` in the hook waits that thread to finish. Finally, this mutual dependence results in the deadlock.

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

Yes, the executor shutdown won't hang.

### How was this patch tested?

Pass existing tests.

Closes #33759 from Ngone51/fix-executor-shutdown-hang.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 996551fece)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-18 22:47:06 +08:00
Kousuke Saruta b749b49a28 [SPARK-36400][SPARK-36398][SQL][WEBUI] Make ThriftServer recognize spark.sql.redaction.string.regex
### What changes were proposed in this pull request?

This PR fixes an issue that ThriftServer doesn't recognize `spark.sql.redaction.string.regex`.
The problem is that sensitive information included in queries can be exposed.
![thrift-password1](https://user-images.githubusercontent.com/4736016/129440772-46379cc5-987b-41ac-adce-aaf2139f6955.png)
![thrift-password2](https://user-images.githubusercontent.com/4736016/129440775-fd328c0f-d128-4a20-82b0-46c331b9fd64.png)

### Why are the changes needed?

Bug fix.

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

No.

### How was this patch tested?

Ran ThriftServer, connect to it and execute `CREATE TABLE mytbl2(a int) OPTIONS(url="jdbc:mysql//example.com:3306", driver="com.mysql.jdbc.Driver", dbtable="test_tbl", user="test_usr", password="abcde");` with `spark.sql.redaction.string.regex=((?i)(?<=password=))(".*")|('.*')`
Then, confirmed UI.

![thrift-hide-password1](https://user-images.githubusercontent.com/4736016/129440863-cabea247-d51f-41a4-80ac-6c64141e1fb7.png)
![thrift-hide-password2](https://user-images.githubusercontent.com/4736016/129440874-96cd0f0c-720b-4010-968a-cffbc85d2be5.png)

Closes #33743 from sarutak/thrift-redact.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
(cherry picked from commit b914ff7d54)
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-08-18 13:32:03 +09:00
Takuya UESHIN 528fca8944 [SPARK-36370][PYTHON][FOLLOWUP] Use LooseVersion instead of pkg_resources.parse_version
### What changes were proposed in this pull request?

This is a follow-up of #33687.

Use `LooseVersion` instead of `pkg_resources.parse_version`.

### Why are the changes needed?

In the previous PR, `pkg_resources.parse_version` was used, but we should use `LooseVersion` instead to be consistent in the code base.

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

No.

### How was this patch tested?

Existing tests.

Closes #33768 from ueshin/issues/SPARK-36370/LooseVersion.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 7fb8ea319e)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-18 10:36:17 +09:00
Wenchen Fan 5107ad3157 [SPARK-36535][SQL] Refine the sql reference doc
### What changes were proposed in this pull request?

Refine the SQL reference doc:
- remove useless subitems in the sidebar
- remove useless sub-menu-pages (e.g. `sql-ref-syntax-aux.md`)
- avoid using `#####` in `sql-ref-literals.md`

### Why are the changes needed?

The subitems in the sidebar are quite useless, as the menu page serves the same functionalities:
<img width="1040" alt="WX20210817-2358402x" src="https://user-images.githubusercontent.com/3182036/129765924-d7e69bc1-e351-4581-a6de-f2468022f372.png">
It's also extra work to keep the manu page and sidebar subitems in sync (The ANSI compliance page is already out of sync).

The sub-menu-pages are only referenced by the sidebar, and duplicates the content of the menu page. As a result, the `sql-ref-syntax-aux.md` is already outdated compared to the menu page. It's easier to just look at the menu page.

The `#####` is not rendered properly:
<img width="776" alt="WX20210818-0001192x" src="https://user-images.githubusercontent.com/3182036/129766760-6f385443-e597-44aa-888d-14d128d45f84.png">
It's better to avoid using it.

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

No

### How was this patch tested?

N/A

Closes #33767 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
(cherry picked from commit 4b015e8d7d)
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-08-17 12:46:49 -07:00
Cedric-Magnan e15daa31b3 [SPARK-36370][PYTHON] _builtin_table directly imported from pandas instead of being redefined
### What changes were proposed in this pull request?
Suggesting to refactor the way the _builtin_table is defined in the `python/pyspark/pandas/groupby.py` module.
Pandas has recently refactored the way we import the _builtin_table and is now part of the pandas.core.common module instead of being an attribute of the pandas.core.base.SelectionMixin class.

### Why are the changes needed?
This change is not fully needed but the current implementation redefines this table within pyspark, so any changes of this table from the pandas library would need to be updated in the pyspark repository as well.

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

### How was this patch tested?
Ran the following command successfully :
```sh
python/run-tests --testnames 'pyspark.pandas.tests.test_groupby'
```
Tests passed in 327 seconds

Closes #33687 from Cedric-Magnan/_builtin_table_from_pandas.

Authored-by: Cedric-Magnan <cedric.magnan@artefact.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 964dfe254f)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-08-17 10:47:01 -07:00
Gengliang Wang 70635b4b26 Revert "[SPARK-35028][SQL] ANSI mode: disallow group by aliases"
### What changes were proposed in this pull request?

Revert [[SPARK-35028][SQL] ANSI mode: disallow group by aliases ](https://github.com/apache/spark/pull/32129)

### Why are the changes needed?

It turns out that many users are using the group by alias feature.  Spark has its precedence rule when alias names conflict with column names in Group by clause: always use the table column. This should be reasonable and acceptable.
Also, external DBMS such as PostgreSQL and MySQL allow grouping by alias, too.

As we are going to announce ANSI mode GA in Spark 3.2, I suggest allowing the group by alias in ANSI mode.

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

No, the feature is not released yet.

### How was this patch tested?

Unit tests

Closes #33758 from gengliangwang/revertGroupByAlias.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 8bfb4f1e72)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-17 20:24:09 +08:00
Max Gekk 07c6976f79 [SPARK-36524][SQL] Common class for ANSI interval types
### What changes were proposed in this pull request?
Add new type `AnsiIntervalType` to `AbstractDataType.scala`, and extend it by `YearMonthIntervalType` and by `DayTimeIntervalType`

### Why are the changes needed?
To improve code maintenance. The change will allow to replace checking of both `YearMonthIntervalType` and `DayTimeIntervalType` by a check of `AnsiIntervalType`, for instance:
```scala
    case _: YearMonthIntervalType | _: DayTimeIntervalType => false
```
by
```scala
    case _: AnsiIntervalType => false
```

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

### How was this patch tested?
By existing test suites.

Closes #33753 from MaxGekk/ansi-interval-type-trait.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 82a31508af)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-08-17 12:28:07 +03:00
attilapiros eb09be9e68 [SPARK-36052][K8S] Introducing a limit for pending PODs
Introducing a limit for pending PODs (newly created/requested executors included).
This limit is global for all the resource profiles. So first we have to count all the newly created and pending PODs (decreased by the ones which requested to be deleted) then we can share the remaining pending POD slots among the resource profiles.

Without this PR dynamic allocation could request too many PODs and the K8S scheduler could be overloaded and scheduling of PODs will be affected by the load.

No.

With new unit tests.

Closes #33492 from attilapiros/SPARK-36052.

Authored-by: attilapiros <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 1dced492fb)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-16 16:06:29 -07:00
Gengliang Wang 41e5144b53 [SPARK-36521][SQL] Disallow comparison between Interval and String
### What changes were proposed in this pull request?

Disallow comparison between Interval and String in the default type coercion rules.

### Why are the changes needed?

If a binary comparison contains interval type and string type, we can't decide which
interval type the string should be promoted as. There are many possible interval
types, such as year interval, month interval, day interval, hour interval, etc.

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

No, the new interval type is not released yet.

### How was this patch tested?

Existing UT

Closes #33750 from gengliangwang/disallowCom.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit 26d6b952dc)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-08-16 22:41:25 +03:00
Yuanjian Li 4caa43e398 [SPARK-36041][SS][DOCS] Introduce the RocksDBStateStoreProvider in the programming guide
### What changes were proposed in this pull request?
Add the document for the new RocksDBStateStoreProvider.

### Why are the changes needed?
User guide for the new feature.

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

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

Closes #33683 from xuanyuanking/SPARK-36041.

Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
(cherry picked from commit 3d57e00a7f)
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-08-16 12:32:19 -07:00
zhuqi-lucas 2fb62e0e3d [SPARK-35548][CORE][SHUFFLE] Handling new attempt has started error message in BlockPushErrorHandler in client
### What changes were proposed in this pull request?
Add a new type of error message in BlockPushErrorHandler which indicates the PushblockStream message is received after a new application attempt has started. This error message should be correctly handled in client without retrying the block push.

### Why are the changes needed?
When we get a block push failure because of the too old attempt, we will not retry pushing the block nor log the exception on the client side.

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

### How was this patch tested?
Add the corresponding unit test.

Closes #33617 from zhuqi-lucas/master.

Authored-by: zhuqi-lucas <821684824@qq.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 05cd5f97c3)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-16 13:59:54 -05:00
Xinrong Meng cb14a32005 [SPARK-36469][PYTHON] Implement Index.map
### What changes were proposed in this pull request?
Implement `Index.map`.

The PR is based on https://github.com/databricks/koalas/pull/2136. Thanks awdavidson for the prototype.

`map` of CategoricalIndex and DatetimeIndex will be implemented in separate PRs.

### Why are the changes needed?
Mapping values using input correspondence (a dict, Series, or function) is supported in pandas as [Index.map](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.Index.map.html).
We shall also support hat.

### Does this PR introduce _any_ user-facing change?
Yes. `Index.map` is available now.

```py
>>> psidx = ps.Index([1, 2, 3])

>>> psidx.map({1: "one", 2: "two", 3: "three"})
Index(['one', 'two', 'three'], dtype='object')

>>> psidx.map(lambda id: "{id} + 1".format(id=id))
Index(['1 + 1', '2 + 1', '3 + 1'], dtype='object')

>>> pser = pd.Series(["one", "two", "three"], index=[1, 2, 3])
>>> psidx.map(pser)
Index(['one', 'two', 'three'], dtype='object')
```

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

Closes #33694 from xinrong-databricks/index_map.

Authored-by: Xinrong Meng <xinrong.meng@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
(cherry picked from commit 4dcd746025)
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2021-08-16 11:06:23 -07:00
Kazuyuki Tanimura 9149cad57d [SPARK-32210][CORE] Fix NegativeArraySizeException in MapOutputTracker with large spark.default.parallelism
### What changes were proposed in this pull request?
The current `MapOutputTracker` class may throw `NegativeArraySizeException` with a large number of partitions. Within the serializeOutputStatuses() method, it is trying to compress an array of mapStatuses and outputting the binary data into (Apache)ByteArrayOutputStream . Inside the (Apache)ByteArrayOutputStream.toByteArray(), negative index exception happens because the index is int and overflows (2GB limit) when the output binary size is too large.

This PR proposes two high-level ideas:
  1. Use `org.apache.spark.util.io.ChunkedByteBufferOutputStream`, which has a way to output the underlying buffer as `Array[Array[Byte]]`.
  2. Change the signatures from `Array[Byte]` to `Array[Array[Byte]]` in order to handle over 2GB compressed data.

### Why are the changes needed?
This issue seems to be missed out in the earlier effort of addressing 2GB limitations [SPARK-6235](https://issues.apache.org/jira/browse/SPARK-6235)

Without this fix, `spark.default.parallelism` needs to be kept at the low number. The drawback of setting smaller spark.default.parallelism is that it requires more executor memory (more data per partition). Setting `spark.io.compression.zstd.level` to higher number (default 1) hardly helps.

That essentially means we have the data size limit that for shuffling and does not scale.

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

### How was this patch tested?
Passed existing tests
```
build/sbt "core/testOnly org.apache.spark.MapOutputTrackerSuite"
```
Also added a new unit test
```
build/sbt "core/testOnly org.apache.spark.MapOutputTrackerSuite  -- -z SPARK-32210"
```
Ran the benchmark using GitHub Actions and didn't not observe any performance penalties. The results are attached in this PR
```
core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt
core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
```

Closes #33721 from kazuyukitanimura/SPARK-32210.

Authored-by: Kazuyuki Tanimura <ktanimura@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 8ee464cd7a)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-16 09:11:51 -07:00
Venkata krishnan Sowrirajan 233af3d239 [SPARK-36374][SHUFFLE][DOC] Push-based shuffle high level user documentation
### What changes were proposed in this pull request?

Document the push-based shuffle feature with a high level overview of the feature and corresponding configuration options for both shuffle server side as well as client side. This is how the changes to the doc looks on the browser ([img](https://user-images.githubusercontent.com/8871522/129231582-ad86ee2f-246f-4b42-9528-4ccd693e86d2.png))

### Why are the changes needed?

Helps users understand the feature

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

Docs

### How was this patch tested?

N/A

Closes #33615 from venkata91/SPARK-36374.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 2270ecf32f)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
2021-08-16 10:25:33 -05:00
Liang-Chi Hsieh 3aa933b162 [SPARK-36465][SS] Dynamic gap duration in session window
### What changes were proposed in this pull request?

This patch supports dynamic gap duration in session window.

### Why are the changes needed?

The gap duration used in session window for now is a static value. To support more complex usage, it is better to support dynamic gap duration which determines the gap duration by looking at the current data. For example, in our usecase, we may have different gap by looking at the certain column in the input rows.

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

Yes, users can specify dynamic gap duration.

### How was this patch tested?

Modified existing tests and new test.

Closes #33691 from viirya/dynamic-session-window-gap.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
(cherry picked from commit 8b8d91cf64)
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-08-16 11:06:16 +09:00
Sean Owen b8c1014e23 Update Spark key negotiation protocol 2021-08-14 09:08:29 -05:00
Huaxin Gao ede1d1e9a7 [SPARK-34952][SQL][FOLLOWUP] Normalize pushed down aggregate col name and group by col name
### What changes were proposed in this pull request?
Normalize pushed down aggregate col names and group by col names ...

### Why are the changes needed?
to handle case sensitive col names

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

### How was this patch tested?
Modify existing test

Closes #33739 from huaxingao/normalize.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 3f8ec0dae4)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-08-13 22:31:30 -07:00
Gengliang Wang c898a940e2 [SPARK-36508][SQL] ANSI type coercion: disallow binary operations between Interval and String literal
### What changes were proposed in this pull request?

If a binary operation contains interval type and string literal, we can't decide which interval type the string literal should be promoted as. There are many possible interval types, such as year interval, month interval, day interval, hour interval, etc.
The related binary operation for Interval contains
- Add
- Subtract
- Comparisions

Note that `Interval Multiple/Divide StringLiteral` is valid as them is not binary operators(the left and right are not of the same type). This PR also add tests for them.

### Why are the changes needed?

Avoid ambiguously implicit casting string literals to interval types.

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

No, the ANSI type coercion is not released yet.

### How was this patch tested?

New tests.

Closes #33737 from gengliangwang/disallowStringAndInterval.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit ecdea91602)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-08-14 10:50:43 +08:00
yi.wu 101f720cc9 [SPARK-32920][CORE][FOLLOW-UP] Fix string interpolator in the log
### What changes were proposed in this pull request?

fix string interpolator

### Why are the changes needed?

To log the correct stage info.

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

No.

### How was this patch tested?

Pass existed tests.

Closes #33738 from Ngone51/fix.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit a47ceaf549)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-13 21:44:30 +09:00
Xingbo Jiang 09a1ddba41 [SPARK-36500][CORE] Fix temp_shuffle file leaking when a task is interrupted
### What changes were proposed in this pull request?

When a task thread is interrupted, the underlying output stream referred by `DiskBlockObjectWriter.mcs` may have been closed, then we get IOException when flushing the buffered data. This breaks the assumption that `revertPartialWritesAndClose()` should not throw exceptions.

To fix the issue, we can catch the IOException in `ManualCloseOutputStream.manualClose()`.

### Why are the changes needed?

Previously the IOException was not captured, thus `revertPartialWritesAndClose()` threw an exception. When this happens, `BypassMergeSortShuffleWriter.stop()` would stop deleting the temp_shuffle files tracked by `partitionWriters`, hens lead to temp_shuffle file leak issues.

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

No, this is an internal bug fix.

### How was this patch tested?

Tested by running a longevity stress test. After the fix, there is no more leaked temp_shuffle files.

Closes #33731 from jiangxb1987/temp_shuffle.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit ec5f3a17e3)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-08-13 19:25:27 +09:00
Gengliang Wang eaf92bea99 [SPARK-36499][SQL][TESTS] Test Interval multiply / divide null
### What changes were proposed in this pull request?

Test the following valid operations:
```
year-month interval * null
null * year-month interval
year-month interval / null
```
and invalid operations:
```
null / interval
int / interval
```

### Why are the changes needed?

Improve test coverage

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

No

### How was this patch tested?

Pass CI

Closes #33729 from gengliangwang/addTest.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
(cherry picked from commit eb6be7f1ee)
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-08-13 11:06:09 +03:00
gengjiaan eb840578f7 [SPARK-36428][SQL] the seconds parameter of make_timestamp should accept integer type
### What changes were proposed in this pull request?
With ANSI mode, `SELECT make_timestamp(1, 1, 1, 1, 1, 1)` fails, because the 'seconds' parameter needs to be of type DECIMAL(8,6), and INT can't be implicitly casted to DECIMAL(8,6) under ANSI mode.

```
org.apache.spark.sql.AnalysisException
cannot resolve 'make_timestamp(1, 1, 1, 1, 1, 1)' due to data type mismatch: argument 6 requires decimal(8,6) type, however, '1' is of int type.; line 1 pos 7
```

We should update the function `make_timestamp` to allow integer type 'seconds' parameter.

### Why are the changes needed?
Make `make_timestamp` could accepts integer as 'seconds' parameter.

### Does this PR introduce _any_ user-facing change?
'Yes'.
`make_timestamp` could accepts integer as 'seconds' parameter.

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

Closes #33665 from beliefer/SPARK-36428.

Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 7d82336734)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-08-13 13:13:15 +08:00