Commit graph

7762 commits

Author SHA1 Message Date
Kousuke Saruta 38bc45b0b5
[SPARK-30654][WEBUI][FOLLOWUP] Remove bootstrap-tooltip.js which is no longer used
### What changes were proposed in this pull request?

This PR removes `bootstrap-tooltip.js` which is no longer used.
That script is replaced with `bootstrap.bundle.min.js` in SPARK-30654 ( #27370 ).

### Why are the changes needed?

For cleaning up repository..

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

No.

### How was this patch tested?

Manually checked whether tooltips are shown in the UI and no error message shown in the debug console.

Closes #28515 from sarutak/remove-tooltipjs.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-13 01:04:57 -07:00
Dongjoon Hyun 07209f3e2d [SPARK-31683][CORE] Make Prometheus output consistent with DropWizard 4.1 result
### What changes were proposed in this pull request?

This PR aims to update Prometheus-related output format to be consistent with DropWizard 4.1 result.
- Add `Number` metrics for gauges metrics.
- Add `type` labels.

### Why are the changes needed?

SPARK-29032 added Prometheus support. After that, SPARK-29674 upgraded DropWizard for JDK9+ support and this caused difference in output labels and number of keys for Guage metrics. The current status is different from Apache Spark 2.4.5. Since we cannot change DropWizard, this PR aims to be consistent in Apache Spark 3.0.0 only.

**DropWizard 3.x**
```
metrics_master_aliveWorkers_Value 1.0
```

**DropWizard 4.1**
```
metrics_master_aliveWorkers_Value{type="gauges",} 1.0
metrics_master_aliveWorkers_Number{type="gauges",} 1.0
```

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

Yes, but this is a new feature in 3.0.0.

### How was this patch tested?

Manually check the output like the following.

**JMXExporter Result**
```
$ curl -s http://localhost:8088/ | grep "^metrics_master" | sort
metrics_master_aliveWorkers_Number{type="gauges",} 1.0
metrics_master_aliveWorkers_Value{type="gauges",} 1.0
metrics_master_apps_Number{type="gauges",} 0.0
metrics_master_apps_Value{type="gauges",} 0.0
metrics_master_waitingApps_Number{type="gauges",} 0.0
metrics_master_waitingApps_Value{type="gauges",} 0.0
metrics_master_workers_Number{type="gauges",} 1.0
metrics_master_workers_Value{type="gauges",} 1.0
```

**This PR**
```
$ curl -s http://localhost:8080/metrics/master/prometheus/ | grep master
metrics_master_aliveWorkers_Number{type="gauges"} 1
metrics_master_aliveWorkers_Value{type="gauges"} 1
metrics_master_apps_Number{type="gauges"} 0
metrics_master_apps_Value{type="gauges"} 0
metrics_master_waitingApps_Number{type="gauges"} 0
metrics_master_waitingApps_Value{type="gauges"} 0
metrics_master_workers_Number{type="gauges"} 1
metrics_master_workers_Value{type="gauges"} 1
```

Closes #28510 from dongjoon-hyun/SPARK-31683.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2020-05-12 19:57:48 +00:00
oleg d7c3e9e53e
[SPARK-31456][CORE] Fix shutdown hook priority edge cases
### What changes were proposed in this pull request?
Fix application order for shutdown hooks for the priorities of Int.MaxValue, Int.MinValue

### Why are the changes needed?
The bug causes out-of-order execution of shutdown hooks if their priorities were Int.MinValue or Int.MaxValue

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

### How was this patch tested?
Added a test covering the change.

Closes #28494 from oleg-smith/SPARK-31456_shutdown_hook_priority.

Authored-by: oleg <oleg@nexla.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-11 13:10:39 -07:00
Dongjoon Hyun b80309bdb4
[SPARK-31674][CORE][DOCS] Make Prometheus metric endpoints experimental
### What changes were proposed in this pull request?

This PR aims to new Prometheus-format metric endpoints experimental in Apache Spark 3.0.0.

### Why are the changes needed?

Although the new metrics are disabled by default, we had better make it experimental explicitly in Apache Spark 3.0.0 since the output format is still not fixed. We can finalize it in Apache Spark 3.1.0.

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

Only doc-change is visible to the users.

### How was this patch tested?

Manually check the code since this is a documentation and class annotation change.

Closes #28495 from dongjoon-hyun/SPARK-31674.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-10 22:32:26 -07:00
Kousuke Saruta 0fb607ef37 [SPARK-30385][WEBUI] WebUI occasionally throw IOException on stop()
### What changes were proposed in this pull request?

This PR added a workaround for the issue which occasionally happens when SparkContext#stop() is called.
I think this issue can occurs on macOS with OpenJDK / OracleJDK 1.8.
If this issue happens, following stack trace is shown.
```
20/05/03 02:17:54 WARN AbstractConnector:
java.io.IOException: No such file or directory
	at sun.nio.ch.NativeThread.signal(Native Method)
	at sun.nio.ch.ServerSocketChannelImpl.implCloseSelectableChannel(ServerSocketChannelImpl.java:292)
	at java.nio.channels.spi.AbstractSelectableChannel.implCloseChannel(AbstractSelectableChannel.java:234)
	at java.nio.channels.spi.AbstractInterruptibleChannel.close(AbstractInterruptibleChannel.java:115)
	at org.eclipse.jetty.server.ServerConnector.close(ServerConnector.java:368)
	at org.eclipse.jetty.server.AbstractNetworkConnector.shutdown(AbstractNetworkConnector.java:105)
	at org.eclipse.jetty.server.Server.doStop(Server.java:439)
	at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
	at org.apache.spark.ui.ServerInfo.stop(JettyUtils.scala:501)
	at org.apache.spark.ui.WebUI.$anonfun$stop$2(WebUI.scala:173)
	at org.apache.spark.ui.WebUI.$anonfun$stop$2$adapted(WebUI.scala:173)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.ui.WebUI.stop(WebUI.scala:173)
	at org.apache.spark.ui.SparkUI.stop(SparkUI.scala:101)
	at org.apache.spark.SparkContext.$anonfun$stop$6(SparkContext.scala:1966)
	at org.apache.spark.SparkContext.$anonfun$stop$6$adapted(SparkContext.scala:1966)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.SparkContext.$anonfun$stop$5(SparkContext.scala:1966)
	at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1357)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1966)
	at org.apache.spark.repl.Main$.$anonfun$doMain$3(Main.scala:79)
	at org.apache.spark.repl.Main$.$anonfun$doMain$3$adapted(Main.scala:79)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.repl.Main$.doMain(Main.scala:79)
	at org.apache.spark.repl.Main$.main(Main.scala:58)
	at org.apache.spark.repl.Main.main(Main.scala)
	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.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```

This issue happens when the Jetty's acceptor thread shrinks before the main thread sends a signal to the thread.

Jetty's acceptor thread waits for a new connection request and blocked by `accept(this.fd, newfd, isaa)` in [`sun.nio.ch.ServerSocketChannelImpl#accept`](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l241).

When `org.eclipse.jetty.server.Server.doStop` is called in the main thread, the thread reaches [this code](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l280).

The server socket descriptor will be closed by `nd.preClose` in the main thread.
Then, `accept()` in acceptor thread throws an Exception due to "Bad file descriptor" in case of macOS.
After the exception is thrown, the acceptor thread will continue to [fetch a task](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L783).
If the thread obtain the `SHRINK` task [here](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L854), the thread will be shrink.
If, the acceptor thread finishes before `NativeThread.signal` is called in the main thread, this issue happens.

I have confirmed this issue happens even `jetty-9.4.28.v20200408`.
Because the stack trace is displayed by the [logger](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-server/src/main/java/org/eclipse/jetty/server/ServerConnector.java#L372), it's difficult to suppress it.
According to [this condition](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L842), shrink doesn't happen if the idle time is 0. So this PR adds a workaround that set the idle time to 0 immediately before stop.

In case of Linux, the acceptor thread is still blocked by `accept` even though `np.preClose` is called in the main thread.
The acceptor thread will return from `accept` when `NativeThread.signal` is called in the main thread.
It seems that the implementation of `accept systemcall` called in `accept` is different between Linux and macOS.
So, I believe this issue doesn't happen on Linux.

Also, the implementation of `NativeThread.signal` is a little bit changed in [OpenJDK 9](http://hg.openjdk.java.net/jdk9/jdk9/jdk/rev/7b17bff2ea36) for macOS.
So this issue doesn't happen for macOS with OpenJDK 9+.

You can reproduce this issue by following instructions using debugger.

1. Launch spark-shell in local mode with JDWP enabled.
2. Access to WebUI. This is needed to increase the number of SparkUI thread to greater than minThreads to meet the condition of shrink.
3. Enable the following breakpoints. Note that don't suspend all threads when a thread reaches one of the breakpoints. Only the threads which reach the line should be suspended.
  3.1 [long now = System.nanoTime(); at org.eclipse.jetty.util.thread.QueuedThreadPool#idleJobPoll](https://github.com/eclipse/jetty.project/blob/jetty-9.4.18.v20190429/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L850)
  3.2 [NativeThread.signal(th); at sun.nio.ch.ServerSocketChannelImpl#implCloseSelectableChannel](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l283)
  3.3 [thread = 0; at ServerSocketChannelImpl#accept](http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/sun/nio/ch/ServerSocketChannelImpl.java#l247)
4. Quit spark-shell.
5.  Waiting for a thread reaching the breakpoint `3.1` and until the following condition become true (The idle time of those threads are 1min and you can confirm it using the expression evaluation feature if your debugger supports ).
`(System.nanoTime() - last) > TimeUnit.MILLISECONDS.toNanos(_idleTimeout)`
6. The acceptor thread named `SparkUI-<N>-acceptor-0` should be suspended at the breakpoint `3.3` so continue this thread. This thread will reach the breakpoint at `3.1` and continue further. Then, the acceptor thread will be shrink.
7. Continue all the threads rest.

### Why are the changes needed?

This stack trace is not brought by Spark but it confuses users.

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

No.

### How was this patch tested?

Tested by the reproduce procedure above and confirmed acceptor thread is no longer shrink.

Closes #28437 from sarutak/SPARK-30385.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-08 08:41:18 +00:00
Dongjoon Hyun 24fac1e0c7
[SPARK-31646][FOLLOWUP][TESTS] Add clean up code and disable irrelevent conf 2020-05-07 17:50:32 -07:00
tianlzhang dad61ed465
[SPARK-31646][SHUFFLE] Remove unused registeredConnections counter from ShuffleMetrics
### What changes were proposed in this pull request?
Remove unused `registeredConnections` counter from `ExternalBlockHandler#ShuffleMetrics`

This was added by SPARK-25642 at 3.0.0
- 8dd29fe36b

### Why are the changes needed?
It's `registeredConnections` counter created in `TransportContext` that's really counting the numbers and it's misleading for people who want to add new metrics like `registeredConnections`.

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

### How was this patch tested?
Add UTs to ensure all expected metrics are registered for `ExternalShuffleService` and `YarnShuffleService`

Closes #28457 from manuzhang/spark-31611-pre.

Lead-authored-by: tianlzhang <tianlzhang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-07 15:22:13 -07:00
Kent Yao bd6b53cc0b [SPARK-31631][TESTS] Fix test flakiness caused by MiniKdc which throws 'address in use' BindException with retry
### What changes were proposed in this pull request?
The `Kafka*Suite`s are flaky because of the Hadoop MiniKdc issue - https://issues.apache.org/jira/browse/HADOOP-12656
> Looking at MiniKdc implementation, if port is 0, the constructor use ServerSocket to find an unused port, assign the port number to the member variable port and close the ServerSocket object; later, in initKDCServer(), instantiate a TcpTransport object and bind at that port.

> It appears that the port may be used in between, and then throw the exception.

Related test failures are suspected,  such as https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/122225/testReport/org.apache.spark.sql.kafka010/KafkaDelegationTokenSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/

```scala
[info] org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite *** ABORTED *** (15 seconds, 426 milliseconds)
[info]   java.net.BindException: Address already in use
[info]   at sun.nio.ch.Net.bind0(Native Method)
[info]   at sun.nio.ch.Net.bind(Net.java:433)
[info]   at sun.nio.ch.Net.bind(Net.java:425)
[info]   at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:223)
[info]   at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74)
[info]   at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:198)
[info]   at org.apache.mina.transport.socket.nio.NioSocketAcceptor.open(NioSocketAcceptor.java:51)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor.registerHandles(AbstractPollingIoAcceptor.java:547)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor.access$400(AbstractPollingIoAcceptor.java:68)
[info]   at org.apache.mina.core.polling.AbstractPollingIoAcceptor$Acceptor.run(AbstractPollingIoAcceptor.java:422)
[info]   at org.apache.mina.util.NamePreservingRunnable.run(NamePreservingRunnable.java:64)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748)
```
After comparing the error stack trace with similar issues reported  in different projects, such as
https://issues.apache.org/jira/browse/KAFKA-3453
https://issues.apache.org/jira/browse/HBASE-14734

We can be sure that they are caused by the same problem issued in HADOOP-12656.

In the PR, We apply the approach from HBASE first before we finally drop Hadoop 2.7.x

### Why are the changes needed?

fix test flakiness

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

### How was this patch tested?

the test itself passing Jenkins

Closes #28442 from yaooqinn/SPARK-31631.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-05-07 14:37:03 +09:00
zhengruifeng ebdf41dd69 [SPARK-30642][ML][PYSPARK] LinearSVC blockify input vectors
### What changes were proposed in this pull request?
1, add new param `blockSize`;
2, add a new class InstanceBlock;
3, **if `blockSize==1`, keep original behavior; if `blockSize>1`, stack input vectors to blocks (like ALS/MLP);**
4, if `blockSize>1`, standardize the input outside of optimization procedure;

### Why are the changes needed?
1, reduce RAM to persist traing dataset; (save about 40% RAM)
2, use Level-2 BLAS routines; (4x ~ 5x faster on dataset `epsilon`)

### Does this PR introduce any user-facing change?
Yes, a new param is added

### How was this patch tested?
existing and added testsuites

Closes #28349 from zhengruifeng/blockify_svc_II.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2020-05-06 10:06:23 +08:00
yi.wu 61a6ca5d3f
[SPARK-31643][TEST] Fix flaky o.a.s.scheduler.BarrierTaskContextSuite.barrier task killed, interrupt
### What changes were proposed in this pull request?

Make sure the task has nearly reached `context.barrier()` before killing.

### Why are the changes needed?

In case of the task is killed before it reaches `context.barrier()`, the task will not create the expected file.

```
Error Message
org.scalatest.exceptions.TestFailedException: new java.io.File(dir, killedFlagFile).exists() was false Expect barrier task being killed.
Stacktrace
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: new java.io.File(dir, killedFlagFile).exists() was false Expect barrier task being killed.
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$testBarrierTaskKilled$1(BarrierTaskContextSuite.scala:266)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$testBarrierTaskKilled$1$adapted(BarrierTaskContextSuite.scala:226)
	at org.apache.spark.SparkFunSuite.withTempDir(SparkFunSuite.scala:163)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.testBarrierTaskKilled(BarrierTaskContextSuite.scala:226)
	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$new$29(BarrierTaskContextSuite.scala:277)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
```

[Here's](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/122273/testReport/org.apache.spark.scheduler/BarrierTaskContextSuite/barrier_task_killed__interrupt/) the full error messages.

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

### How was this patch tested?

Closes #28454 from Ngone51/fix_kill_interrupt.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-05 12:36:42 -07:00
Akshat Bordia c71198ab6c [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver
### What changes were proposed in this pull request?
Fixing an issue where Spark Master UI Fails to load if the application is waiting for workers to launch driver.

**Root Cause:**
This is happening due to the fact that the submitted application is waiting for a worker to be free to run the driver. Due to this resource is set to null in the formatResourcesAddresses method and this is running into null pointer exception.
![image](https://user-images.githubusercontent.com/31816865/80801557-77ee9300-8bca-11ea-92b7-b8df58b68de3.png)

**Fix:**
Added a null check before forming a resource address and display "None" if the driver isn't launched yet.

### Why are the changes needed?

Spark Master UI should load as expected when applications are waiting for workers to run driver.

### Does this PR introduce _any_ user-facing change?
The worker column in Spark Master UI will show "None" if the driver hasn't been launched yet.
![image](https://user-images.githubusercontent.com/31816865/80801671-be43f200-8bca-11ea-86c3-381925f82cc7.png)

### How was this patch tested?
Tested on a local setup. Launched 2 applications and ensured that Spark Master UI loads fine.
![image](https://user-images.githubusercontent.com/31816865/80801883-5b9f2600-8bcb-11ea-8a1a-cc597aabc4c2.png)

Closes #28429 from akshatb1/MasterUIBug.

Authored-by: Akshat Bordia <akshat.bordia31@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-05-05 08:58:37 -05:00
Dongjoon Hyun 0907f2e7b5 [SPARK-27963][FOLLOW-UP][DOCS][CORE] Remove for testing because CleanerListener is used ExecutorMonitor during dynamic allocation
### What changes were proposed in this pull request?

This PR aims to remove `for testing` from `CleanerListener` class description to promote this private class more clearly.

### Why are the changes needed?

After SPARK-27963 (Allow dynamic allocation without a shuffle service), `CleanerListener` is used in `ExecutorMonitor` during dynamic allocation. Specifically, `CleanerListener.shuffleCleaned` is used.
- https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala#L385-L392

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

No.

### How was this patch tested?

This is a private doc-only change.

Closes #28452 from dongjoon-hyun/SPARK-MINOR.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-05 10:07:30 +09:00
Tianshi Zhu a222644e1d [SPARK-31267][SQL] Flaky test: WholeStageCodegenSparkSubmitSuite.Generated code on driver should not embed platform-specific constant
### What changes were proposed in this pull request?

Allow customized timeouts for `runSparkSubmit`, which will make flaky tests more likely to pass by using a larger timeout value.

I was able to reproduce the test failure on my laptop, which took 1.5 - 2 minutes to finish the test. After increasing the timeout, the test now can pass locally.

### Why are the changes needed?

This allows slow tests to use a larger timeout, so they are more likely to succeed.

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

No

### How was this patch tested?

The test was able to pass on my local env after the change.

Closes #28438 from tianshizz/SPARK-31267.

Authored-by: Tianshi Zhu <zhutianshirea@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-04 14:50:38 +09:00
Xingbo Jiang b7cde42b04 [SPARK-31619][CORE] Rename config "spark.dynamicAllocation.shuffleTimeout" to "spark.dynamicAllocation.shuffleTracking.timeout"
### What changes were proposed in this pull request?
The "spark.dynamicAllocation.shuffleTimeout" configuration only takes effect if "spark.dynamicAllocation.shuffleTracking.enabled" is true, so we should re-namespace that configuration so that it's nested under the "shuffleTracking" one.

### How was this patch tested?
Covered by current existing test cases.

Closes #28426 from jiangxb1987/confName.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 11:46:17 +09:00
Weichen Xu ee1de66fe4 [SPARK-31549][PYSPARK] Add a develop API invoking collect on Python RDD with user-specified job group
### What changes were proposed in this pull request?
I add a new API in pyspark RDD class:

def collectWithJobGroup(self, groupId, description, interruptOnCancel=False)

This API do the same thing with `rdd.collect`, but it can specify the job group when do collect.
The purpose of adding this API is, if we use:

```
sc.setJobGroup("group-id...")
rdd.collect()
```
The `setJobGroup` API in pyspark won't work correctly. This related to a bug discussed in
https://issues.apache.org/jira/browse/SPARK-31549

Note:

This PR is a rather temporary workaround for `PYSPARK_PIN_THREAD`, and as a step to migrate to  `PYSPARK_PIN_THREAD` smoothly. It targets Spark 3.0.

- `PYSPARK_PIN_THREAD` is unstable at this moment that affects whole PySpark applications.
- It is impossible to make it runtime configuration as it has to be set before JVM is launched.
- There is a thread leak issue between Python and JVM. We should address but it's not a release blocker for Spark 3.0 since the feature is experimental. I plan to handle this after Spark 3.0 due to stability.

Once `PYSPARK_PIN_THREAD` is enabled by default, we should remove this API out ideally. I will target to deprecate this API in Spark 3.1.

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

### Does this PR introduce any user-facing change?
A develop API in pyspark: `pyspark.RDD. collectWithJobGroup`

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

Closes #28395 from WeichenXu123/collect_with_job_group.

Authored-by: Weichen Xu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 10:08:16 +09:00
Kousuke Saruta 91ec2eacfa
[SPARK-31565][WEBUI] Unify the font color of label among all DAG-viz
### What changes were proposed in this pull request?

This PR unifies the font color of label as `#333333` among all DAG-viz.

### Why are the changes needed?

For the consistent appearance among all DAG-viz.
There are three types of DAG-viz in the WebUI.
One is for stages, another one is for RDDs and the last one is for query plans.
But the font color of labels are slightly different among them.

For stages, the color is `#333333` (simply 333) which is specified by `spark-dag-viz.css`.
<img width="355" alt="job-graph" src="https://user-images.githubusercontent.com/4736016/80321397-b517f580-8857-11ea-8c8e-cf68f648ab05.png">
<img width="310" alt="job-graph-color" src="https://user-images.githubusercontent.com/4736016/80321399-ba754000-8857-11ea-8708-83bdef4bc1d1.png">

For RDDs, the color is `#212529` which is specified by `bootstrap.min.js`.
<img width="386" alt="stage-graph" src="https://user-images.githubusercontent.com/4736016/80321438-f0b2bf80-8857-11ea-9c2a-13fa0fd1431c.png">
<img width="313" alt="stage-graph-color" src="https://user-images.githubusercontent.com/4736016/80321444-fa3c2780-8857-11ea-81b2-4f1203d47896.png">

For query plans, the color is `black` which is specified by `spark-sql-viz.css`.
<img width="449" alt="plan-graph" src="https://user-images.githubusercontent.com/4736016/80321490-61f27280-8858-11ea-9c3a-2c98d3d4d03b.png">
<img width="316" alt="plan-graph-color" src="https://user-images.githubusercontent.com/4736016/80321496-6ae34400-8858-11ea-8fe8-0d6e4a821608.png">

After the change, the appearance is like as follows (no change for stages).

For RDDs.
<img width="389" alt="stage-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321613-6b300f00-8859-11ea-912f-d92474aa9f47.png">

For query plans.
<img width="456" alt="plan-graph-fixed" src="https://user-images.githubusercontent.com/4736016/80321638-9a468080-8859-11ea-974c-33c56a8ffe1a.png">

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

Yes. The unified color is slightly lighter than ever.

### How was this patch tested?

Confirmed that the color code among all DAG-viz are `#333333` using browser's debug console.

Closes #28352 from sarutak/unify-label-color.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-26 16:57:23 -07:00
Kousuke Saruta d61c6219cd [SPARK-31534][WEBUI] Text for tooltip should be escaped
### What changes were proposed in this pull request?

This PR escapes text for tooltip for DAG Viz and Timeline View.

### Why are the changes needed?

This is a bug.
Normally, DAG Viz and Timeline View show tooltip like as follows.

<img width="278" alt="dag-viz-tooltip" src="https://user-images.githubusercontent.com/4736016/80127481-5a6c6880-85cf-11ea-8daf-cfd59aa3ba09.png">
<img width="477" alt="timeline-tooltip" src="https://user-images.githubusercontent.com/4736016/80127500-60624980-85cf-11ea-9b0f-cce301019e3a.png">

They contain a callsite properly.
However, if a callsite contains characters which should be escaped for HTML without escaping , the corresponding tooltips wouldn't show the callsite and its following text properly.
<img width="179" alt="dag-viz-tooltip-before-fixed" src="https://user-images.githubusercontent.com/4736016/80128480-b1267200-85d0-11ea-8035-ad68ae5fbcab.png">
<img width="261" alt="timeline-tooltip-before-fixed" src="https://user-images.githubusercontent.com/4736016/80128492-b5eb2600-85d0-11ea-9556-c48490110244.png">

The reason of this issue is that the source texts of the tooltip texts are not escaped.

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

No.

### How was this patch tested?

I tested manually.
First, I ran a job `sc.parallelize(1 to 10).collect` in Spark Shell then, visited AllJobsPage and JobPage and confirmed tooltip texts.
<img width="196" alt="dag-viz-tooltip-fixed" src="https://user-images.githubusercontent.com/4736016/80128813-2db95080-85d1-11ea-82f8-90a1f4547f30.png">
<img width="363" alt="timeline-tooltip-fixed" src="https://user-images.githubusercontent.com/4736016/80128824-31e56e00-85d1-11ea-9818-492b72b1c56e.png">

I also added a testcase.

Closes #28317 from sarutak/fix-tooltip.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-04-27 05:14:46 +09:00
yi.wu ab8cada1f9
[SPARK-31521][CORE] Correct the fetch size when merging blocks into a merged block
### What changes were proposed in this pull request?

Fix the wrong fetch size.

### Why are the changes needed?

The fetch size should be the sum of the size of merged block and the total size of those merging blocks. But we missed the size of merged block.

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

No.

### How was this patch tested?

Added a regression test.

Closes #28301 from Ngone51/fix_merged_block_size.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-24 22:11:35 -07:00
Holden Karau 9faad07ce7 HOTFIX Revert "[SPARK-20732][CORE] Decommission cache blocks
HOTFIX test issue introduced in SPARK-20732

Closes #28337 from holdenk/revert-SPARK-20732.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-04-24 18:51:25 -07:00
Prakhar Jain 249b214590 [SPARK-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned
### What changes were proposed in this pull request?
After changes in SPARK-20628, CoarseGrainedSchedulerBackend can decommission an executor and stop assigning new tasks on it. We should also decommission the corresponding blockmanagers in the same way. i.e. Move the cached RDD blocks from those executors to other active executors.

### Why are the changes needed?
We need to gracefully decommission the block managers so that the underlying RDD cache blocks are not lost in case the executors are taken away forcefully after some timeout (because of spotloss/pre-emptible VM etc). Its good to save as much cache data as possible.

Also In future once the decommissioning signal comes from Cluster Manager (say YARN/Mesos etc), dynamic allocation + this change gives us opportunity to downscale the executors faster by making the executors free of cache data.

Note that this is a best effort approach. We try to move cache blocks from decommissioning executors to active executors. If the active executors don't have free resources available on them for caching, then the decommissioning executors will keep the cache block which it was not able to move and it will still be able to serve them.

Current overall Flow:

1. CoarseGrainedSchedulerBackend receives a signal to decommissionExecutor. On receiving the signal, it do 2 things - Stop assigning new tasks (SPARK-20628), Send another message to BlockManagerMasterEndpoint (via BlockManagerMaster) to decommission the corresponding BlockManager.

2. BlockManagerMasterEndpoint receives "DecommissionBlockManagers" message. On receiving this, it moves the corresponding block managers to "decommissioning" state. All decommissioning BMs are excluded from the getPeers RPC call which is used for replication. All these decommissioning BMs are also sent message from BlockManagerMasterEndpoint to start decommissioning process on themselves.

3. BlockManager on worker (say BM-x) receives the "DecommissionBlockManager" message. Now it will start BlockManagerDecommissionManager thread to offload all the RDD cached blocks. This thread can make multiple reattempts to decommission the existing cache blocks (multiple reattempts might be needed as there might not be sufficient space in other active BMs initially).

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

### How was this patch tested?
Added UTs.

Closes #27864 from prakharjain09/SPARK-20732-rddcache-1.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-04-24 11:22:08 -07:00
yi.wu 263f04db86 [SPARK-31485][CORE] Avoid application hang if only partial barrier tasks launched
### What changes were proposed in this pull request?

Use `dagScheduler.taskSetFailed` to abort a barrier stage instead of throwing exception within `resourceOffers`.

### Why are the changes needed?

Any non fatal exception thrown within Spark RPC framework can be swallowed:

100fc58da5/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala (L202-L211)

 The method `TaskSchedulerImpl.resourceOffers` is also within the scope of Spark RPC framework. Thus, throw exception inside `resourceOffers` won't fail the application.

 As a result, if a barrier stage fail the require check at `require(addressesWithDescs.size == taskSet.numTasks, ...)`, the barrier stage will fail the check again and again util all tasks from `TaskSetManager` dequeued.   But since the barrier stage isn't really executed, the application will hang.

The issue can be reproduced by the following test:

```scala
initLocalClusterSparkContext(2)
val rdd0 = sc.parallelize(Seq(0, 1, 2, 3), 2)
val dep = new OneToOneDependency[Int](rdd0)
val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq("executor_h_0"),Seq("executor_h_0")))
rdd.barrier().mapPartitions { iter =>
  BarrierTaskContext.get().barrier()
  iter
}.collect()
```

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

Yes, application hang previously but fail-fast after this fix.

### How was this patch tested?

Added a regression test.

Closes #28257 from Ngone51/fix_barrier_abort.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-24 04:17:06 +00:00
bmarcott f093480af9
fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)
fix method return type doc
2020-04-23 14:38:15 -05:00
Antonin Delpeuch 497024956a
[SPARK-31518][CORE] Expose filterByRange in JavaPairRDD
### What changes were proposed in this pull request?

This exposes the `filterByRange` method from `OrderedRDDFunctions` in the Java API (as a method of JavaPairRDD).

This is the only method of `OrderedRDDFunctions` which is not exposed in the Java API so far.

### Why are the changes needed?

This improves the consistency between the Scala and Java APIs. Calling the Scala method manually from a Java context is cumbersome as it requires passing many ClassTags.

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

Yes, a new method in the Java API.

### How was this patch tested?

With unit tests. The implementation of the Scala method is already tested independently and it was not touched in this PR.

Suggesting srowen as a reviewer.

Closes #28293 from wetneb/SPARK-31518.

Authored-by: Antonin Delpeuch <antonin@delpeuch.eu>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-22 20:04:17 -07:00
Thomas Graves 95aec091e4 [SPARK-29641][PYTHON][CORE] Stage Level Sched: Add python api's and tests
### What changes were proposed in this pull request?

As part of the Stage level scheduling features, add the Python api's to set resource profiles.
This also adds the functionality to properly apply the pyspark memory configuration when specified in the ResourceProfile. The pyspark memory configuration is being passed in the task local properties. This was an easy way to get it to the PythonRunner that needs it. I modeled this off how the barrier task scheduling is passing the addresses. As part of this I added in the JavaRDD api's because those are needed by python.

### Why are the changes needed?

python api for this feature

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

Yes adds the java and python apis for user to specify a ResourceProfile to use stage level scheduling.

### How was this patch tested?

unit tests and manually tested on yarn. Tests also run to verify it errors properly on standalone and local mode where its not yet supported.

Closes #28085 from tgravescs/SPARK-29641-pr-base.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-23 10:20:39 +09:00
Nicholas Marcott 8b77b31835 [SPARK-18886][CORE][FOLLOWUP] allow follow up locality resets even if no task was launched
### What changes were proposed in this pull request?
Remove the requirement to launch a task in order to reset locality wait timer.

### Why are the changes needed?
Recently https://github.com/apache/spark/pull/27207 was merged, but contained a bug which leads to undesirable behavior.

The crux of the issue is that single resource offers couldn't reset the timer, if there had been a previous reject followed by an allResourceOffer with no available resources.
This lead to a problem where once locality level reached ANY, single resource offers are all accepted, leading allResourceOffers to be left with no resources to utilize (hence no task being launched on an all resource offer -> no timer reset). The task manager would be stuck in ANY locality level.

Noting down here the downsides of using below reset conditions, in case we want to follow up.
As this is quite complex, I could easily be missing something, so please comment/respond if you have more bad behavior scenarios or find something wrong here:
The format is:

> **Reset condition**
>  - the unwanted side effect
>      - the cause/use case

Below references to locality increase/decrease mean:
```
PROCESS_LOCAL, NODE_LOCAL ... .. ANY
    ------ locality decrease --->
   <----- locality increase -----
```

**Task launch:**
- locality decrease:
   - Blacklisting, FAIR/FIFO scheduling, or task resource requirements can minimize tasks launched
 - locality increase:
   - single task launch decreases locality despite many tasks remaining

**No delay schedule reject since last allFreeResource offer**
- locality decrease:
   - locality wait less than allFreeResource offer frequency, which occurs at least 1 per second
- locality increase:
   - single resource (or none) not rejected despite many tasks remaining (other lower priority tasks utilizing resources)

**Current impl - No delay schedule reject since last (allFreeResource offer + task launch)**
- locality decrease:
  - all from above
- locality increase:
   - single resource accepted and task launched despite many tasks remaining

The current impl is an improvement on the legacy (task launch) in that unintended locality decrease case is similar and the unintended locality increase case only occurs when the cluster is fully utilized.

For the locality increase cases, perhaps a config which specifies a certain % of tasks in a taskset to finish before resetting locality levels would be helpful.

**If** that was considered a good approach then perhaps removing the task launch as a requirement would eliminate most of downsides listed above.
Lemme know if you have more ideas for eliminating locality increase downside of **No delay schedule reject since last allFreeResource offer**

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

### How was this patch tested?
TaskSchedulerImplSuite

Also manually tested similar to how I tested in https://github.com/apache/spark/pull/27207 using [this simple app](https://github.com/bmarcott/spark-test-apps/blob/master/src/main/scala/TestLocalityWait.scala).

With the new changes, given locality wait of 10s the behavior is generally:
10 seconds of locality being respected, followed by a single full utilization of resources using ANY locality level, followed by 10 seconds of locality being respected, and so on

If the legacy flag is enabled (spark.locality.wait.legacyResetOnTaskLaunch=true), the behavior is only scheduling PROCESS_LOCAL tasks (only utilizing a single executor)

cloud-fan
tgravescs

Closes #28188 from bmarcott/nmarcott-locality-fix.

Authored-by: Nicholas Marcott <481161+bmarcott@users.noreply.github.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-22 08:25:24 -05:00
Liang-Chi Hsieh 1d30884963 [SPARK-31484][CORE][FLOLLOWUP] Use taskAttemptId in checkpoint filename
### What changes were proposed in this pull request?

As suggested by https://github.com/apache/spark/pull/28255#discussion_r412619438, this patch proposes to use taskAttemptId in checkpoint filename, instead of stageAttemptNumber + attemptNumber.

### Why are the changes needed?

To simplify checkpoint simplified and unique.

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

No

### How was this patch tested?

Existing tests.

Closes #28289 from viirya/SPARK-31484-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-22 21:23:48 +09:00
herman cf6038499d
[SPARK-31511][SQL] Make BytesToBytesMap iterators thread-safe
### What changes were proposed in this pull request?
This PR increases the thread safety of the `BytesToBytesMap`:
- It makes the `iterator()` and `destructiveIterator()` methods used their own `Location` object. This used to be shared, and this was causing issues when the map was being iterated over in two threads by two different iterators.
- Removes the `safeIterator()` function. This is not needed anymore.
- Improves the documentation of a couple of methods w.r.t. thread-safety.

### Why are the changes needed?
It is unexpected an iterator shares the object it is returning with all other iterators. This is a violation of the iterator contract, and it causes issues with iterators over a map that are consumed in different threads.

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

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

Closes #28286 from hvanhovell/SPARK-31511.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 18:17:19 -07:00
Huang-Yi a5c16cbf05
[SPARK-31233][CORE] Enhance RpcTimeoutException Log Message
### What changes were proposed in this pull request?

askAbortable method throw TimeoutException while it does no complete in time. Currently, the error message contains null as remoteAddr when receiver is in client mode.
This change is to print out correct rpcAddress instead of null in the error message.

### Why are the changes needed?

It provides the address of an endpoint which does not reply in time. It helps users to find slow executors when timeout happens.

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

No.

### How was this patch tested?

Add a unit test.

Closes #28002 from Huang-yi-3456/SPARK-31233-enhance-rpctimeoutexception-log.

Authored-by: Huang-Yi <huang.yi.3456@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-21 14:08:37 -07:00
yi.wu 7103f19fea [SPARK-31472][CORE] Make sure Barrier Task always return messages or exception with abortableRpcFuture check
### What changes were proposed in this pull request?

Rewrite the periodically check logic of  `abortableRpcFuture` to make sure that barrier task would always return either desired messages or expected exception.

This PR also simplify a bit around `AbortableRpcFuture`.

### Why are the changes needed?

Currently, the periodically check logic of  `abortableRpcFuture` is done by following:

```scala
...
var messages: Array[String] = null

while (!abortableRpcFuture.toFuture.isCompleted) {
   messages = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second)
   ...
}
return messages
```
It's possible that `abortableRpcFuture` complete before next invocation on `messages = ...`. In this case, the task may return null messages or execute successfully while it should throw exception(e.g. `SparkException` from `BarrierCoordinator`).

And here's a flaky test which caused by this bug:

```
[info] BarrierTaskContextSuite:
[info] - share messages with allGather() call *** FAILED *** (18 seconds, 705 milliseconds)
[info]   org.apache.spark.SparkException: Job aborted due to stage failure: Could not recover from a failed barrier ResultStage. Most recent failure reason: Stage failed because barrier task ResultTask(0, 2) finished unsuccessfully.
[info] java.lang.NullPointerException
[info] 	at scala.collection.mutable.ArrayOps$ofRef$.length$extension(ArrayOps.scala:204)
[info] 	at scala.collection.mutable.ArrayOps$ofRef.length(ArrayOps.scala:204)
[info] 	at scala.collection.IndexedSeqOptimized.toList(IndexedSeqOptimized.scala:285)
[info] 	at scala.collection.IndexedSeqOptimized.toList$(IndexedSeqOptimized.scala:284)
[info] 	at scala.collection.mutable.ArrayOps$ofRef.toList(ArrayOps.scala:198)
[info] 	at org.apache.spark.scheduler.BarrierTaskContextSuite.$anonfun$new$4(BarrierTaskContextSuite.scala:68)
...
```

The test exception can be reproduced by changing the line `messages = ...` to the following:

```scala
messages = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 10.micros)
Thread.sleep(5000)
```

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

No.

### How was this patch tested?

Manually test and update some unit tests.

Closes #28245 from Ngone51/fix_barrier.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-21 10:12:56 +00:00
Onur Satici ad965103a5
[SPARK-30949][K8S][CORE] Decouple requests and parallelism on drivers in K8s
### What changes were proposed in this pull request?
`spark.driver.cores` configuration is used to set the amount of parallelism in kubernetes cluster mode drivers. Previously the amount of parallelism in the drivers were the number of cores in the host when running on JDK 8u120 or older, or the maximum of driver containers resource requests and limits when running on [JDK 8u121 or newer](https://bugs.openjdk.java.net/browse/JDK-8173345). This will enable users to specify `spark.driver.cores` to set parallelism, and specify `spark.kubernetes.driver.requests.cores` to limit the resource requests of the driver container, effectively decoupling the two

### Why are the changes needed?
Drivers submitted in kubernetes cluster mode set the parallelism of various components like `RpcEnv`, `MemoryManager`, `BlockManager` from inferring the number of available cores by calling `Runtime.getRuntime().availableProcessors()`. By using this, spark applications running on JDK 8u120 or older incorrectly get the total number of cores in the host, [ignoring the cgroup limits set by kubernetes](https://bugs.openjdk.java.net/browse/JDK-6515172). JDK 8u121 and newer runtimes do not have this problem.

Orthogonal to this, it is currently not possible to decouple resource limits on the driver container with the amount of parallelism of the various network and memory components listed above.

### Does this PR introduce any user-facing change?
Yes. Previously the amount of parallelism in kubernetes cluster mode submitted drivers were the number of cores in the host when running on JDK 8u120 or older, or the maximum of driver containers resource requests and limits when running on JDK 8u121 or newer. Now the value of `spark.driver.cores` is used.

### How was this patch tested?
happy to add tests if my proposal looks reasonable

Closes #27695 from onursatici/os/decouple-requests-and-parallelism.

Authored-by: Onur Satici <onursatici@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-20 21:32:43 -07:00
Liang-Chi Hsieh e3ac56c8f4
[SPARK-31484][CORE] Add stage attempt number to temp checkpoint filename to avoid file already existing exception
### What changes were proposed in this pull request?

To avoid file already existing exception when creating checkpoint file, this PR proposes to add stage attempt number to temporary checkpoint file.

### Why are the changes needed?

On our production clusters, we have seen checkpointing failure. The failed stage can possibly leave partial written checkpoint file, the task of retried stage to write checkpoint file could fail due to`FileAlreadyExistsException` when creating the same file, like
```
org.apache.hadoop.fs.FileAlreadyExistsException: /path_to_checkpoint/rdd-114/.part-03154-attempt-0 for client xxx.xxx.xxx.xxx already exists
org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.startFile(FSDirWriteFileOp.java:359)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2353)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2273)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:728)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:413)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:447)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:989)
	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:851)
	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:794)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2490)

	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 org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:121)
	at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:88)
	at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:270)
	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1263)
	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1205)
	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:473)
	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:470)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:470)
	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:411)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:929)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:910)
	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:872)
	at org.apache.spark.rdd.ReliableCheckpointRDD$.writePartitionToCheckpointFile(ReliableCheckpointRDD.scala:204)
```

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

Yes. Users won't see checkpoint file already existing exception after this PR.

### How was this patch tested?

Add unit test.

Closes #28255 from viirya/delete-temp-checkpoint.

Authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-19 09:11:17 -07:00
beliefer 1513673f83 [SPARK-30913][SPARK-30841][CORE][SQL][FOLLOWUP] Supplement version information to the configuration of Tests.scala and SQL
### What changes were proposed in this pull request?
I checked all the config of Spark again. find some new commit not add version information.

**Test.scala**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.testing.skipValidateCores | 3.1.0 | SPARK-29154 | 474b1bb5c2bce2f83c4dd8e19b9b7c5b3aebd6c4#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  

**SQL**
Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.legacy.integerGroupingId | 3.1.0 | SPARK-30279 | 71c73d58f6e88d2558ed2e696897767d93bac60f#diff-9a6b543db706f1a90f790783d6930a13 |  

The two config only exists in branch master.

### Why are the changes needed?
Supplement version information.

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

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

Closes #28233 from beliefer/sql-conf-version-legacy-integerGroupingId.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-17 17:10:48 +09:00
yi.wu 40f9dbb628 [SPARK-31425][SQL][CORE] UnsafeKVExternalSorter/VariableLengthRowBasedKeyValueBatch should also respect UnsafeAlignedOffset
### What changes were proposed in this pull request?

Make `UnsafeKVExternalSorter` / `VariableLengthRowBasedKeyValueBatch ` also respect `UnsafeAlignedOffset` when reading the record and update some out of date comemnts.

### Why are the changes needed?

Since `BytesToBytesMap` respects `UnsafeAlignedOffset` when writing the record, `UnsafeKVExternalSorter` should also respect `UnsafeAlignedOffset` when reading the record from `BytesToBytesMap` otherwise it will causes data correctness issue.

Unlike `UnsafeKVExternalSorter` may reading records from `BytesToBytesMap`, `VariableLengthRowBasedKeyValueBatch` writes and reads records by itself. Thus, similar to #22053 and [comment](https://github.com/apache/spark/pull/22053#issuecomment-411975239) there, fix for `VariableLengthRowBasedKeyValueBatch` more likely an improvement for the support of SPARC platform.

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

No.

### How was this patch tested?

Manually tested `HashAggregationQueryWithControlledFallbackSuite` with `UAO_SIZE=8`  to simulate SPARC platform. And tests only pass with this fix.

Closes #28195 from Ngone51/fix_uao.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-17 04:48:27 +00:00
yi.wu b2e9e1717b [SPARK-31344][CORE] Polish implementation of barrier() and allGather()
### What changes were proposed in this pull request?

1. Combine  `BarrierRequestToSync` and `AllGatherRequestToSync` into `RequestToSync`, which is distinguished by `RequestMethod` type.

2. Remove unnecessary Json serialization/deserialization

3. Clean up some codes to make runBarrier() and `BarrierCoordinator` more general

4. Remove unused imports.

### Why are the changes needed?

To make codes simpler for better maintain in the future.

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

No.

### How was this patch tested?

This is pure code refactor, so should be covered by existed tests.

Closes #28117 from Ngone51/refactor_barrier.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-04-16 21:23:32 -07:00
Kousuke Saruta 8608189335
[SPARK-31446][WEBUI] Make html elements for a paged table possible to have different id attribute
### What changes were proposed in this pull request?

This PR makes each id attribute for page navigations in a page unique.

`PagedTable#pageNavigation` returns HTML elements representing a page navigation for a paged table.
In the current implementation, the method generates an id and it's used for id attribute for a set of elements for the page navigation.
But some pages have two page navigations so there are two set of elements where corresponding elements have the same id.
For example, there are two `form-completedJob-table-page` id in JobsPage.
### Why are the changes needed?

Each id attribute should be unique in a page.
The following is a screenshot of warning messages shown with Chrome when I visit JobsPage (Firefox doesn't show in my environment).
<img width="1440" alt="warning-jobspage" src="https://user-images.githubusercontent.com/4736016/79261523-f3fa9280-7eca-11ea-861d-d54f04f1b0bc.png">

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

No.

### How was this patch tested?

I added a test case for `pageNavigation` extended.
I also manually tested that there were no warning messages for the uniqueness in JobsPage and JobPage.

Closes #28217 from sarutak/unique-form-id.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-16 16:24:11 -07:00
Kousuke Saruta df27350142 [SPARK-31420][WEBUI][FOLLOWUP] Make locale of timeline-view 'en'
<!--
Thanks for sending a pull request!  Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
  4. Be sure to keep the PR description updated to reflect all changes.
  5. Please write your PR title to summarize what this PR proposes.
  6. If possible, provide a concise example to reproduce the issue for a faster review.
  7. If you want to add a new configuration, please read the guideline first for naming configurations in
     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
-->

### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
  2. If you fix some SQL features, you can provide some references of other DBMSes.
  3. If there is design documentation, please add the link.
  4. If there is a discussion in the mailing list, please add the link.
-->
This change explicitly set locale of timeline view to 'en' to be the same appearance as before upgrading vis-timeline.

### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
  1. If you propose a new API, clarify the use case for a new API.
  2. If you fix a bug, you can clarify why it is a bug.
-->
We upgraded vis-timeline in #28192 and the upgraded version is different from before we used in the notation of dates.
The notation seems to be dependent on locale. The following is appearance in my Japanese environment.
<img width="557" alt="locale-changed" src="https://user-images.githubusercontent.com/4736016/79265314-de886700-7ed0-11ea-8641-fa76b993c0d9.png">

Although the notation is in Japanese, the default format is a little bit unnatural (e.g. 4月9日 05:39 is natural rather than 9 四月 05:39).

I found we can get the same appearance as before by explicitly set locale to 'en'.

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
No.

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
I visited JobsPage, JobPage and StagePage and confirm that timeline view shows dates with 'en' locale.
<img width="735" alt="fix-date-appearance" src="https://user-images.githubusercontent.com/4736016/79267107-8bfc7a00-7ed3-11ea-8a25-f6681d04a83c.png">

NOTE: #28192 will be backported to branch-2.4 and branch-3.0 so this PR should be follow #28214 and #28213 .

Closes #28218 from sarutak/fix-locale-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2020-04-17 02:31:08 +09:00
Kousuke Saruta 04f04e0ea7 [SPARK-31420][WEBUI] Infinite timeline redraw in job details page
### What changes were proposed in this pull request?

Upgrade vis.js to fix an infinite re-drawing issue.

As reported here, old releases of vis.js have that issue.
Fortunately, the latest version seems to resolve the issue.

With the latest release of vis.js, there are some performance issues with the original `timeline-view.js` and `timeline-view.css` so I also changed them.

### Why are the changes needed?

For better UX.

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

No. Appearance and functionalities are not changed.

### How was this patch tested?

I confirmed infinite redrawing doesn't happen with a JobPage which I had reproduced the issue.

With the original version of vis.js, I reproduced the issue with the following conditions.

* Use history server and load core/src/test/resources/spark-events.
* Visit the JobPage for job2 in application_1553914137147_0018.
* Zoom out to 80% on Safari / Chrome / Firefox.

Maybe, it depends on OS and the version of browsers.

Closes #28192 from sarutak/upgrade-visjs.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-13 23:23:00 -07:00
Gengliang Wang 28e1a4fa93 [SPARK-31411][UI] Show submitted time and duration in job details page
### What changes were proposed in this pull request?

Show submitted time and duration of a job in its details page

### Why are the changes needed?

When we check job details from the SQL execution page, it will be more convenient if we can get the submission time and duration from the job page, instead of finding the info from job list page.

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

Yes. After changes, the job details page shows the submitted time and duration.

### How was this patch tested?

Manual check
![image](https://user-images.githubusercontent.com/1097932/78974997-0a1de280-7ac8-11ea-8072-ce7a001b1b0c.png)

Closes #28179 from gengliangwang/addSubmittedTimeAndDuration.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-04-13 17:12:26 -07:00
Dongjoon Hyun a6e6fbf2ca
[SPARK-31422][CORE] Fix NPE when BlockManagerSource is used after BlockManagerMaster stops
### What changes were proposed in this pull request?

This PR (SPARK-31422) aims to return empty result in order to avoid `NullPointerException` at `getStorageStatus` and `getMemoryStatus` which happens after `BlockManagerMaster` stops. The empty result is consistent with the current status of `SparkContext` because `BlockManager` and `BlockManagerMaster` is already stopped.

### Why are the changes needed?

In `SparkEnv.stop`, the following stop sequence is used and `metricsSystem.stop` invokes `sink.stop`.
```
blockManager.master.stop()
metricsSystem.stop() --> sinks.foreach(_.stop)
```

However, some sink can invoke `BlockManagerSource` and ends up with `NullPointerException` because `BlockManagerMaster` is already stopped and `driverEndpoint` became `null`.
```
java.lang.NullPointerException
at org.apache.spark.storage.BlockManagerMaster.getStorageStatus(BlockManagerMaster.scala:170)
at org.apache.spark.storage.BlockManagerSource$$anonfun$10.apply(BlockManagerSource.scala:63)
at org.apache.spark.storage.BlockManagerSource$$anonfun$10.apply(BlockManagerSource.scala:63)
at org.apache.spark.storage.BlockManagerSource$$anon$1.getValue(BlockManagerSource.scala:31)
at org.apache.spark.storage.BlockManagerSource$$anon$1.getValue(BlockManagerSource.scala:30)
```

Since `SparkContext` registers and forgets `BlockManagerSource` without deregistering, we had better avoid `NullPointerException` inside `BlockManagerMaster` preventively.
```scala
_env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager))
```

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

Yes. This will remove NPE for the users who uses `BlockManagerSource`.

### How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #28187 from dongjoon-hyun/SPARK-31422.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-11 08:27:30 -07:00
Dongjoon Hyun c6ea6933e2 [SPARK-18886][CORE][TESTS][FOLLOWUP] Fix a test failure due to InvalidUseOfMatchersException
### What changes were proposed in this pull request?

This fixes one UT failure.
```
[info] - extra resources from executor *** FAILED *** (218 milliseconds)
[info]   org.mockito.exceptions.misusing.InvalidUseOfMatchersException: Invalid use of argument matchers!
[info] 0 matchers expected, 1 recorded:
```

### Why are the changes needed?

The original PR was merged with an outdated Jenkins result (7 days before the merging).

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

No.

### How was this patch tested?

Pass the Jenkins or manually do the following.
```
$ build/sbt "core/testOnly *.CoarseGrainedSchedulerBackendSuite"
```

Closes #28174 from dongjoon-hyun/SPARK-18886.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-10 12:02:41 +09:00
Nicholas Marcott 8b4862953a [SPARK-18886][CORE] Make Locality wait time measure resource under utilization due to delay scheduling
### What changes were proposed in this pull request?

[Delay scheduling](http://elmeleegy.com/khaled/papers/delay_scheduling.pdf) is an optimization that sacrifices fairness for data locality in order to improve cluster and workload throughput.

One useful definition of "delay" here is how much time has passed since the TaskSet was using its fair share of resources.

However it is impractical to calculate this delay, as it would require running simulations assuming no delay scheduling. Tasks would be run in different orders with different run times.

Currently the heuristic used to estimate this delay is the time since a task was last launched for a TaskSet. The problem is that it essentially does not account for resource utilization, potentially leaving the cluster heavily underutilized.

This PR modifies the heuristic in an attempt to move closer to the useful definition of delay above.
The newly proposed delay is the time since a TasksSet last launched a task **and** did not reject any resources due to delay scheduling when offered its "fair share".

See the last comments of #26696 for more discussion.

### Why are the changes needed?

cluster can become heavily underutilized as described in [SPARK-18886](https://issues.apache.org/jira/browse/SPARK-18886?jql=project%20%3D%20SPARK%20AND%20text%20~%20delay)

### How was this patch tested?

TaskSchedulerImplSuite

cloud-fan
tgravescs
squito

Closes #27207 from bmarcott/nmarcott-fulfill-slots-2.

Authored-by: Nicholas Marcott <481161+bmarcott@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-09 11:00:29 +00:00
yi.wu a2789c2a51 [SPARK-31379][CORE][TEST] Fix flaky o.a.s.scheduler.CoarseGrainedSchedulerBackendSuite.extra resources from executor
### What changes were proposed in this pull request?

This PR (SPARK-31379) adds one line `when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(Seq.empty)` to avoid allocating resources.

### Why are the changes needed?

The test is flaky and here's part of error stack:

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException:
The code passed to eventually never returned normally. Attempted 325 times over 5.01070979 seconds.
Last failure message: ArrayBuffer("1", "3") did not equal Array("0", "1", "3").
...
org.apache.spark.scheduler.CoarseGrainedSchedulerBackendSuite.eventually(CoarseGrainedSchedulerBackendSuite.scala:45)
```

You can check [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120786/testReport/org.apache.spark.scheduler/CoarseGrainedSchedulerBackendSuite/extra_resources_from_executor/) for details.

And it is flaky because:  after sending `StatusUpdate` to `CoarseGrainedSchedulerBackend`, `CoarseGrainedSchedulerBackend` will call `makeOffer` immediately once releasing the resources. So, it's possible that `availableAddrs` has allocated again before we assert `execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")`.

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

No.

### How was this patch tested?

The issue can be stably reproduced by inserting `Thread.sleep(3000)` after the line of sending `StatusUpdate`. After applying this fix, the issue is gone.

Closes #28145 from Ngone51/fix_flaky.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-08 17:54:28 +09:00
Thomas Graves 30f1866078
[SPARK-31378][CORE] stage level scheduling dynamic allocation bug with initial num executors
### What changes were proposed in this pull request?

I found a bug in the stage level scheduling dynamic allocation code when you have a non default profile and it has an initial number of executors the same as what the number of executors needed for the first job, then we don't properly request the executors.  This causes a hang.

The issue is that when a new stage is added and the initial number of executors is set, we set the target to be the initial number.  Unfortunately that makes the code in the update and sync function think it has already requested that number.  So to fix this, when there is an initial number we just go ahead and request executors at that point. This is basically what happens on startup to handle the case with the default profile.

### Why are the changes needed?

bug

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

no

### How was this patch tested?

unit test and manually test on yarn cluster. Went though multiple scenarios initial numbers, minimum number and number executor required by the first stage.

Closes #28146 from tgravescs/SPARK-31378.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 14:06:19 -07:00
Holden Karau 8f010bd0a8
[SPARK-31208][CORE] Add an expiremental cleanShuffleDependencies
### What changes were proposed in this pull request?

Add a cleanShuffleDependencies as an experimental developer feature to allow folks to clean up shuffle files more aggressively than we currently do.

### Why are the changes needed?

Dynamic scaling on Kubernetes (introduced in Spark 3) depends on only shutting down executors without shuffle files. However Spark does not aggressively clean up shuffle files (see SPARK-5836) and instead depends on JVM GC on the driver to trigger deletes. We already have a mechanism to explicitly clean up shuffle files from the ALS algorithm where we create a lot of quickly orphaned shuffle files. We should expose this as an advanced developer feature to enable people to better clean-up shuffle files improving dynamic scaling of their jobs on Kubernetes.

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

This adds a new experimental API.

### How was this patch tested?

ALS already used a mechanism like this, re-targets the ALS code to the new interface, tested with existing ALS tests.

Closes #28038 from holdenk/SPARK-31208-allow-users-to-cleanup-shuffle-files.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-07 13:54:36 -07:00
Kousuke Saruta f5250a581b
[SPARK-31360][WEBUI] Fix hung-up issue in StagePage
### What changes were proposed in this pull request?

This change (SPARK-31360) fixes a hung-up issue in StagePage.
StagePage will be hung-up with following operations.

1. Run a job with shuffle.
`scala> sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _).collect`

2. Visit StagePage for the stage writing shuffle data and check `Shuffle Write Time`.
<img width="401" alt="check-shuffle-write-time" src="https://user-images.githubusercontent.com/4736016/78557730-4513e200-784c-11ea-8b42-a5053b9489a5.png">

3. Run a job with no shuffle.
`scala> sc.parallelize(1 to 10).collect`

4. Visit StagePage for the last stage.
<img width="956" alt="hungup" src="https://user-images.githubusercontent.com/4736016/78557746-4f35e080-784c-11ea-83e8-5db745b88535.png">

This issue is caused by following reason.

In stagepage.js, an array `optionalColumns` has indices for columns for optional metrics.
If a stage doesn't perform shuffle read or write, the corresponding indices are removed from the array.
StagePage doesn't try to create column for such metrics, even if the state of corresponding optional metrics are preserved as "visible".
But, if a stage doesn't perform both shuffle read and write, the index for `Shuffle Write Time` isn't removed.
In that case, StagePage tries to create a column for `Shuffle Write Time` even though there are no metrics for shuffle write, leading hungup.

### Why are the changes needed?

This is a bug.

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

No.

### How was this patch tested?

I tested with operations I explained above and confirmed that StagePage won't be hung-up.

Closes #28136 from sarutak/fix-ui-hungup.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-06 12:38:12 -07:00
yi.wu 5d76b12e9b
[SPARK-29154][FOLLOW-UP][CORE] RDD.resourceProfile should not be serialized
### What changes were proposed in this pull request?

Mark `RDD.resourceProfile` as `transient`.

### Why are the changes needed?

`RDD.resourceProfile` should only be used at driver side.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #28108 from Ngone51/spark_29154_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-03 09:41:03 -07:00
yi.wu a4fc6a6e98 [SPARK-31249][CORE] Fix flaky CoarseGrainedSchedulerBackendSuite.custom log url for Spark UI is applied
### What changes were proposed in this pull request?

In `CoarseGrainedSchedulerBackendSuite.RegisterExecutor`, change it to post `SparkListenerExecutorAdded` before `context.reply(true)`.

### Why are the changes needed?

To fix flaky `CoarseGrainedSchedulerBackendSuite.custom log url for Spark UI is applied`.

In this test, though we use `askSync` to register executor but `askSync` could be finished before we posting the 3rd `SparkListenerExecutorAdded` event to the listener bus due to the reason that `context.reply(true)` comes before `listenerBus.post`.

The error can be reproduced if you:
- loop it for 500 times in one turn
- or, insert a `Thread.sleep(1000)` between `post` and `reply`.

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

No.

### How was this patch tested?

Loop the flaky tests for 1000 times without any error.

Closes #28100 from Ngone51/fix_spark_31249.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-03 16:05:07 +09:00
Thomas Graves 55dea9be62 [SPARK-29153][CORE] Add ability to merge resource profiles within a stage with Stage Level Scheduling
### What changes were proposed in this pull request?

For the stage level scheduling feature, add the ability to optionally merged resource profiles if they were specified on multiple RDD within a stage.  There is a config to enable this feature, its off by default (spark.scheduler.resourceProfile.mergeConflicts). When the config is set to true, Spark will merge the profiles selecting the max value of each resource (cores, memory, gpu, etc).  further documentation will be added with SPARK-30322.

This also added in the ability to check if an equivalent resource profile already exists. This is so that if a user is running stages and combining the same profiles over and over again we don't get an explosion in the number of profiles.

### Why are the changes needed?

To allow users to specify resource on multiple RDD and not worry as much about if they go into the same stage and fail.

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

Yes, when the config is turned on it now merges the profiles instead of errorring out.

### How was this patch tested?

Unit tests

Closes #28053 from tgravescs/SPARK-29153.

Lead-authored-by: Thomas Graves <tgraves@apache.org>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-02 08:30:18 -05:00
turbofei ec28925236 [SPARK-31179] Fast fail the connection while last connection failed in fast fail time window
## What changes were proposed in this pull request?

For TransportFactory, the requests sent to the same address share a clientPool.
Specially, when the io.numConnectionPerPeer is 1, these requests would share a same client.
When this address is unreachable, the createClient operation would be still timeout.
And these requests would block each other during createClient, because there is a lock for this shared client.
It would cost connectionNum \* connectionTimeOut \* maxRetry to retry, and then fail the task.

It fact, it is expected that this task could fail in connectionTimeOut * maxRetry.

In this PR, I set a fastFail time window for the clientPool, if the last connection failed in this time window, the new connection would fast fail.

## Why are the changes needed?
It can save time for some cases.
## Does this PR introduce any user-facing change?
No.
## How was this patch tested?
Existing UT.

Closes #27943 from turboFei/SPARK-31179-fast-fail-connection.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-04-02 08:18:14 -05:00
ulysses 2c0e15e1d0
[SPARK-31285][CORE] uppercase schedule mode string at config
### What changes were proposed in this pull request?

In `TaskSchedulerImpl`, Spark will upper schedule mode `SchedulingMode.withName(schedulingModeConf.toUpperCase(Locale.ROOT))`.
But at other place, Spark does not. Such as [AllJobsPage](5945d46c11/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala (L304)).
We should have the same behavior and uppercase schema mode string at config.

### Why are the changes needed?

Before this pr, it's ok to set `spark.scheduler.mode=fair` logically.
But Spark will throw warn log
```
java.util.NoSuchElementException: No value found for 'fair'
	at scala.Enumeration.withName(Enumeration.scala:124)
	at org.apache.spark.ui.jobs.AllJobsPage$$anonfun$22.apply(AllJobsPage.scala:314)
	at org.apache.spark.ui.jobs.AllJobsPage$$anonfun$22.apply(AllJobsPage.scala:314)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.ui.jobs.AllJobsPage.render(AllJobsPage.scala:314)
	at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:90)
	at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:90)
	at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
```

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

Almost no.

### How was this patch tested?

Exists Test.

Closes #28049 from ulysses-you/SPARK-31285.

Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-01 11:46:41 -07:00
Liang-Chi Hsieh 20fc6fa839
[SPARK-31308][PYSPARK] Merging pyFiles to files argument for Non-PySpark applications
### What changes were proposed in this pull request?

This PR (SPARK-31308) proposed to add python dependencies even it is not Python applications.

### Why are the changes needed?

For now, we add `pyFiles` argument to `files` argument only for Python applications, in SparkSubmit. Like the reason in #21420, "for some Spark applications, though they're a java program, they require not only jar dependencies, but also python dependencies.", we need to add `pyFiles` to `files` even it is not Python applications.

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

Yes. After this change, for non-PySpark applications, the Python files specified by `pyFiles` are also added to `files` like PySpark applications.

### How was this patch tested?

Manually test on jupyter notebook or do `spark-submit` with `--verbose`.

```
Spark config:
...
(spark.files,file:/Users/dongjoon/PRS/SPARK-PR-28077/a.py)
(spark.submit.deployMode,client)
(spark.master,local[*])
```

Closes #28077 from viirya/pyfile.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-31 18:08:55 -07:00
yi.wu 5ec1814e22
[SPARK-31248][CORE][TEST] Fix flaky ExecutorAllocationManagerSuite.interleaving add and remove
### What changes were proposed in this pull request?

This PR (SPARK-31248) uses `ManualClock` to disable `ExecutorAllocationManager.schedule()`  in order to avoid unexpected update of target executors.

### Why are the changes needed?

`ExecutorAllocationManager` will call `schedule` periodically, which may update target executors before we checking 496f6ac860/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala (L864)

And fail the check:

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 12 did not equal 8
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.ExecutorAllocationManagerSuite.$anonfun$new$51(ExecutorAllocationManagerSuite.scala:864)
	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)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:151)
	at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
```

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

No.

### How was this patch tested?

Update test.

Closes #28084 from Ngone51/spark_31248.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-31 17:26:58 -07:00
Yuanjian Li 07c50784d3 [SPARK-31314][CORE] Revert SPARK-29285 to fix shuffle regression caused by creating temporary file eagerly
### What changes were proposed in this pull request?
This reverts commit 8cf76f8d61. #25962

### Why are the changes needed?
In SPARK-29285, we change to create shuffle temporary eagerly. This is helpful for not to fail the entire task in the scenario of occasional disk failure. But for the applications that many tasks don't actually create shuffle files, it caused overhead. See the below benchmark:
Env: Spark local-cluster[2, 4, 19968], each queries run 5 round, each round 5 times.
Data: TPC-DS scale=99 generate by spark-tpcds-datagen
Results:
|     | Base                                                                                        | Revert                                                                                      |
|-----|---------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------|
| Q20 | Vector(4.096865667, 2.76231748, 2.722007606, 2.514433591, 2.400373579)  Median 2.722007606  | Vector(3.763185446, 2.586498463, 2.593472842, 2.320522846, 2.224627274)  Median 2.586498463 |
| Q33 | Vector(5.872176321, 4.854397586, 4.568787136, 4.393378146, 4.423996818)  Median 4.568787136 | Vector(5.38746785, 4.361236877, 4.082311276, 3.867206824, 3.783188024)  Median 4.082311276  |
| Q52 | Vector(3.978870321, 3.225437871, 3.282411608, 2.869674887, 2.644490664)  Median 3.225437871 | Vector(4.000381522, 3.196025108, 3.248787619, 2.767444508, 2.606163423)  Median 3.196025108 |
| Q56 | Vector(6.238045133, 4.820535173, 4.609965579, 4.313509894, 4.221256227)  Median 4.609965579 | Vector(6.241611339, 4.225592467, 4.195202502, 3.757085755, 3.657525982)  Median 4.195202502 |

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

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

Closes #28072 from xuanyuanking/SPARK-29285-revert.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-31 19:01:08 +08:00
Kengo Seki 0b237bd615 [SPARK-31292][CORE][SQL] Replace toSet.toSeq with distinct for readability
### What changes were proposed in this pull request?

This PR replaces the method calls of `toSet.toSeq` with `distinct`.

### Why are the changes needed?

`toSet.toSeq` is intended to make its elements unique but a bit verbose. Using `distinct` instead is easier to understand and improves readability.

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

No

### How was this patch tested?

Tested with the existing unit tests and found no problem.

Closes #28062 from sekikn/SPARK-31292.

Authored-by: Kengo Seki <sekikn@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-03-29 08:48:08 +09:00
yi.wu 33f532a9f2
[SPARK-31259][CORE] Fix log message about fetch request size in ShuffleBlockFetcherIterator
### What changes were proposed in this pull request?

Fix incorrect log of `cureRequestSize`.

### Why are the changes needed?

In batch mode, `curRequestSize` can be the total size of several block groups. And each group should have its own request size instead of using the total size.

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

No.

### How was this patch tested?

It's only affect log.

Closes #28028 from Ngone51/fix_curRequestSize.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-26 09:11:13 -07:00
Thomas Graves 474b1bb5c2 [SPARK-29154][CORE] Update Spark scheduler for stage level scheduling
### What changes were proposed in this pull request?

This is the core scheduler changes to support Stage level scheduling.

The main changes here include modification to the DAGScheduler to look at the ResourceProfiles associated with an RDD and have those applied inside the scheduler.
Currently if multiple RDD's in a stage have conflicting ResourceProfiles we throw an error. logic to allow this will happen in SPARK-29153. I added the interfaces to RDD to add and get the REsourceProfile so that I could add unit tests for the scheduler. These are marked as private for now until we finish the feature and will be exposed in SPARK-29150. If you think this is confusing I can remove those and remove the tests and add them back later.
I modified the task scheduler to make sure to only schedule on executor that exactly match the resource profile. It will then check those executors to make sure the current resources meet the task needs before assigning it.  In here I changed the way we do the custom resource assignment.
Other changes here include having the cpus per task passed around so that we can properly account for them. Previously we just used the one global config, but now it can change based on the ResourceProfile.
I removed the exceptions that require the cores to be the limiting resource. With this change all the places I found that used executor cores /task cpus as slots has been updated to use the ResourceProfile logic and look to see what resource is limiting.

### Why are the changes needed?

Stage level sheduling feature

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

No

### How was this patch tested?

unit tests and lots of manual testing

Closes #27773 from tgravescs/SPARK-29154.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-26 09:46:36 -05:00
Xingbo Jiang a03fbfbdd5 [SPARK-31207][CORE] Ensure the total number of blocks to fetch equals to the sum of local/hostLocal/remote blocks
### What changes were proposed in this pull request?

Assert the number of blocks to fetch equals the number of local blocks + the number of hostLocal blocks + the number of remote blocks in ShuffleBlockFetcherIterator. Also refactor the code a bit to make it easier to follow.

### Why are the changes needed?

When the numbers don't match it means something is going wrong, we should fail fast.

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

No. This is basically code refactoring.

### How was this patch tested?

Tested with existing test suites.

Closes #27972 from jiangxb1987/BlockFetcher.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-25 13:19:43 +08:00
Xingbo Jiang c2c5b2df50 [SPARK-31239][CORE][TEST] Increase await duration in WorkerDecommissionSuite.verify a task with all workers decommissioned succeeds
### What changes were proposed in this pull request?

The test case has been flaky because the execution time sometimes exceeds the await duration. Increase the await duration to avoid flakiness.

### How was this patch tested?

Tested locally and it didn't fail anymore.

Closes #28007 from jiangxb1987/DecomTest.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-25 13:43:35 +09:00
Kousuke Saruta 88864c0615 [SPARK-31161][WEBUI] Refactor the on-click timeline action in streagming-page.js
### What changes were proposed in this pull request?

Refactor `streaming-page.js` by making on-click timeline action customizable.

### Why are the changes needed?

In the current implementation, `streaming-page.js` is used from Streaming page and Structured Streaming page but the implementation of the on-click timeline action is strongly dependent on Streamng page.
Structured Streaming page doesn't define the on-click action for now but it's better to remove the dependncy for the future.

Originally, I make this change to fix `SPARK-31128` but #27883 resolved it.
So, now this is just for refactoring.

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

No.

### How was this patch tested?

Manual tests with following code and confirmed there are no regression and no error in the debug console in Firefox.

For Structured Streaming:
```
spark.readStream.format("socket").options(Map("host"->"localhost", "port"->"8765")).load.writeStream.format("console").start
```
And then, visited Structured Streaming page and there were no error in the debug console when I clicked a point in the timeline.

For Spark Streaming:
```
import org.apache.spark.streaming._
val ssc = new StreamingContext(sc, Seconds(1))
ssc.socketTextStream("localhost", 8765)
dstream.foreachRDD(rdd => rdd.foreach(println))
ssc.start
```
And then, visited Streaming page and confirmed scrolling down and hilighting work well and there were no error in the debug console when I clicked a point in the timeline.

Closes #27921 from sarutak/followup-SPARK-29543-fix-oncick.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-03-24 13:00:46 -05:00
beliefer ae0699d4b5 [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 |
spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |  
spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 |
spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |  
spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |  
spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |  
spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 |
spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |  
spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |  
spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 |
spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |  
spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |  
spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |  
spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |  
spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |  
spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |  
spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |  
spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 |
spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 |
spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 |
spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |  
spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |  
spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |  
spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |  
spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d |
spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |  
spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |  
spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |  
spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |  
spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |  
spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |  
spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |  
spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 |
spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |  
spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27931 from beliefer/add-version-to-core-config-part-four.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 11:07:43 +09:00
sarthfrey-db 6fd3138e9c [SPARK-30667][CORE] Change BarrierTaskContext allGather method return type
This PR proposes that we change the return type of the `BarrierTaskContext.allGather` method to `Array[String]` instead of `ArrayBuffer[String]` since it is immutable. Based on discussion in #27640. cc zhengruifeng srowen

Closes #27951 from sarthfrey/all-gather-api.

Authored-by: sarthfrey-db <sarth.frey@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-19 12:12:39 +09:00
Adam Binford 9f27a5495d
[SPARK-30860][CORE] Use FileSystem.mkdirs to avoid umask at rolling event log folder and appStatusFile creation
### What changes were proposed in this pull request?
This pull request fixes an issue with rolling event logs. The rolling event log directory is created ignoring the dfs umask setting. This allows the history server to prune old rolling logs when run as the group owner of the event log folder.

### Why are the changes needed?
For non-rolling event logs, log files are created ignoring the umask setting by calling setPermission after creating the file. The default umask of 022 currently causes rolling log directories to be created without group write permissions, preventing the history server from pruning logs of applications not run as the same user as the history server. This adds the same behavior for rolling event logs so users don't need to worry about the umask setting causing different behavior.

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

### How was this patch tested?
Manually. The folder is created with the correct 770 permission. The status file is still affected by the umask setting, but that doesn't stop the folder from being deleted by the history server. I'm not sure if that causes any other issues. I'm not sure how to test something involving a Hadoop setting.

Closes #27764 from Kimahriman/bug/rolling-log-permissions.

Authored-by: Adam Binford <adam.binford@radiantsolutions.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-17 11:20:10 -07:00
Pedro Rossi ed06d98044
[SPARK-25355][K8S] Add proxy user to driver if present on spark-submit
### What changes were proposed in this pull request?

This PR adds the proxy user on the spark-submit command to the childArgs, so the proxy user can be retrieved and used in the KubernetesAplication to add the proxy user in the driver container args

### Why are the changes needed?

The proxy user when used on the spark submit doesn't work on the Kubernetes environment since it doesn't add the `--proxy-user` argument on the driver container and when I added it manually to the Pod definition it worked just fine.

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

No.

### How was this patch tested?

Tests were added

Closes #27422 from PedroRossi/SPARK-25355.

Authored-by: Pedro Rossi <pgrr@cin.ufpe.br>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-16 21:53:58 -07:00
LantaoJin 08bdc9c9b2 [SPARK-31068][SQL] Avoid IllegalArgumentException in broadcast exchange
### What changes were proposed in this pull request?
Fix the IllegalArgumentException in broadcast exchange when numRows over 341 million but less than 512 million.

Since the maximum number of keys that `BytesToBytesMap` supports is 1 << 29, and only 70% of the slots can be used before growing in `HashedRelation`, So here the limitation should not be greater equal than 341 million (1 << 29 / 1.5(357913941)) instead of 512 million.

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

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

Closes #27828 from LantaoJin/SPARK-31068.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Alan Jin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-03-15 20:20:23 -05:00
beliefer f4cd7495f1 [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up #27847 and https://github.com/apache/spark/pull/27852.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 |
spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |  
spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |  
spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |  
spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |  
spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |  
spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49 |  
spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |  
spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |  
spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT
spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |  
spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |  
spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |  
spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |
spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |  
spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |  
spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 |
spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |  
spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |  
spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 |
spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |  
spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |  
spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |  
spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |  
spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |  
spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e |
spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT
spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |  
spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a |
spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |  
spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |  
spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |  
spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |  
spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |  
spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b |
spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |  
spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |  
spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |  
spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27913 from beliefer/add-version-to-core-config-part-three.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-16 10:08:07 +09:00
Dale Clarke 2a4fed0443 [SPARK-30654][WEBUI] Bootstrap4 WebUI upgrade
### What changes were proposed in this pull request?
Spark's Web UI is using an older version of Bootstrap (v. 2.3.2) for the portal pages. Bootstrap 2.x was moved to EOL in Aug 2013 and Bootstrap 3.x was moved to EOL in July 2019 (https://github.com/twbs/release). Older versions of Bootstrap are also getting flagged in security scans for various CVEs:

https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-72889
https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-173700
https://snyk.io/vuln/npm:bootstrap:20180529
https://snyk.io/vuln/npm:bootstrap:20160627

I haven't validated each CVE, but it would be nice to resolve any potential issues and get on a supported release.

The bad news is that there have been quite a few changes between Bootstrap 2 and Bootstrap 4. I've tried updating the library, refactoring/tweaking the CSS and JS to maintain a similar appearance and functionality, and testing the UI for functionality and appearance. This is a fairly large change so I'm sure additional testing and fixes will be needed.

### How was this patch tested?
This has been manually tested, but there is a ton of functionality and there are many pages and detail pages so it is very possible bugs introduced from the upgrade were missed. Additional testing and feedback is welcomed. If it appears a whole page was missed let me know and I'll take a pass at addressing that page/section.

Closes #27370 from clarkead/bootstrap4-core-upgrade.

Authored-by: Dale Clarke <a.dale.clarke@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-13 15:24:48 -07:00
Gengliang Wang 0f463258c2 [SPARK-31128][WEBUI] Fix Uncaught TypeError in streaming statistics page
### What changes were proposed in this pull request?

There is a minor issue in https://github.com/apache/spark/pull/26201
In the streaming statistics page, there is such error
```
streaming-page.js:211 Uncaught TypeError: Cannot read property 'top' of undefined
at SVGCircleElement.<anonymous> (streaming-page.js:211)
at SVGCircleElement.__onclick (d3.min.js:1)
```
in the console after clicking the timeline graph.
![image](https://user-images.githubusercontent.com/1097932/76479745-14b26280-63ca-11ea-9079-0065321795f9.png)

This PR is to fix it.
### Why are the changes needed?

Fix the error of javascript execution.

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

No, the error shows up in the console.

### How was this patch tested?

Manual test.

Closes #27883 from gengliangwang/fixSelector.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-12 20:01:17 -07:00
Gabor Somogyi 231e65092f [SPARK-30874][SQL] Support Postgres Kerberos login in JDBC connector
### What changes were proposed in this pull request?
When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.

This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.

In this PR I've added Postgres support (other supported databases will come in later PRs).

What this PR contains:
* Added `keytab` and `principal` JDBC options
* Added `ConnectionProvider` trait and it's impementations:
  * `BasicConnectionProvider` => unsecure connection
  * `PostgresConnectionProvider` => postgres secure connection
* Added `ConnectionProvider` tests
* Added `PostgresKrbIntegrationSuite` docker integration test
* Created `SecurityUtils` to concentrate re-usable security related functionalities
* Documentation

### Why are the changes needed?
Missing JDBC kerberos support.

### Does this PR introduce any user-facing change?
Yes, 2 additional JDBC options added:
* keytab
* principal

If both provided then Spark does kerberos authentication.

### How was this patch tested?
To demonstrate the functionality with a standalone application I've created this repository: https://github.com/gaborgsomogyi/docker-kerberos

* Additional + existing unit tests
* Additional docker integration test
* Test on cluster manually
* `SKIP_API=1 jekyll build`

Closes #27637 from gaborgsomogyi/SPARK-30874.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-03-12 19:04:35 -07:00
Jungtaek Lim (HeartSaVioR) 3946b24328
[SPARK-31011][CORE] Log better message if SIGPWR is not supported while setting up decommission
### What changes were proposed in this pull request?

This patch changes to log better message (at least relevant to decommission) when registering signal handler for SIGPWR fails. SIGPWR is non-POSIX and not all unix-like OS support it; we can easily find the case, macOS.

### Why are the changes needed?

Spark already logs message on failing to register handler for SIGPWR, but the error message is too general which doesn't give the information of the impact. End users should be noticed that failing to register handler for SIGPWR effectively "disables" the feature of decommission.

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

No.

### How was this patch tested?

Manually tested via running standalone master/worker in macOS 10.14.6, with `spark.worker.decommission.enabled= true`, and submit an example application to run executors.

(NOTE: the message may be different a bit, as the message can be updated in review phase.)

For worker log:

```
20/03/06 17:19:13 INFO Worker: Registering SIGPWR handler to trigger decommissioning.
20/03/06 17:19:13 INFO SignalUtils: Registering signal handler for PWR
20/03/06 17:19:13 WARN SignalUtils: Failed to register SIGPWR - disabling worker decommission.
java.lang.IllegalArgumentException: Unknown signal: PWR
        at java.base/jdk.internal.misc.Signal.<init>(Signal.java:148)
        at jdk.unsupported/sun.misc.Signal.<init>(Signal.java:139)
        at org.apache.spark.util.SignalUtils$.$anonfun$registerSignal$1(SignalUtils.scala:95)
        at scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86)
        at org.apache.spark.util.SignalUtils$.registerSignal(SignalUtils.scala:93)
        at org.apache.spark.util.SignalUtils$.register(SignalUtils.scala:81)
        at org.apache.spark.deploy.worker.Worker.<init>(Worker.scala:73)
        at org.apache.spark.deploy.worker.Worker$.startRpcEnvAndEndpoint(Worker.scala:887)
        at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:855)
        at org.apache.spark.deploy.worker.Worker.main(Worker.scala)
```

For executor:

```
20/03/06 17:21:52 INFO CoarseGrainedExecutorBackend: Registering PWR handler.
20/03/06 17:21:52 INFO SignalUtils: Registering signal handler for PWR
20/03/06 17:21:52 WARN SignalUtils: Failed to register SIGPWR - disabling decommission feature.
java.lang.IllegalArgumentException: Unknown signal: PWR
        at java.base/jdk.internal.misc.Signal.<init>(Signal.java:148)
        at jdk.unsupported/sun.misc.Signal.<init>(Signal.java:139)
        at org.apache.spark.util.SignalUtils$.$anonfun$registerSignal$1(SignalUtils.scala:95)
        at scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86)
        at org.apache.spark.util.SignalUtils$.registerSignal(SignalUtils.scala:93)
        at org.apache.spark.util.SignalUtils$.register(SignalUtils.scala:81)
        at org.apache.spark.executor.CoarseGrainedExecutorBackend.onStart(CoarseGrainedExecutorBackend.scala:86)
        at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:120)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:203)
        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.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:834)
```

Closes #27832 from HeartSaVioR/SPARK-31011.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-11 20:27:00 -07:00
beliefer bd2b3f9132 [SPARK-30911][CORE][DOC] Add version information to the configuration of Status
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Status`.
2.Update the docs of `Status`.
3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |  
spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |  
spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT
spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |  
spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |  
spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27848 from beliefer/add-version-to-status-config.

Lead-authored-by: beliefer <beliefer@163.com>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 11:03:47 +09:00
beliefer c1b2675f2e [SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core
### What changes were proposed in this pull request?
This PR follows up https://github.com/apache/spark/pull/27847.
I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d |
spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |  
spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |  
spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |  
spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |  
spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |  
spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |  
Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled
spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |  
spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 |
spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |  
spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |  
spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |  
spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b |
spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |  
spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 |
spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |  
spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27852 from beliefer/add-version-to-core-config-part-two.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-12 09:52:20 +09:00
Thomas Graves e807118eef
[SPARK-31055][DOCS] Update config docs for shuffle local host reads to have dep on external shuffle service
### What changes were proposed in this pull request?

with SPARK-27651 we now support host local reads for shuffle, but only when external shuffle service is enabled. Update the config docs to state that.

### Why are the changes needed?

clarify dependency

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

no

### How was this patch tested?

n/a

Closes #27812 from tgravescs/SPARK-27651-follow.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-09 12:17:59 -07:00
yi.wu ef51ff9dc8 [SPARK-31082][CORE] MapOutputTrackerMaster.getMapLocation should handle last mapIndex correctly
### What changes were proposed in this pull request?

In `getMapLocation`, change the condition from `...endMapIndex < statuses.length` to `...endMapIndex <= statuses.length`.

### Why are the changes needed?

`endMapIndex` is exclusive, we should include it when comparing to `statuses.length`. Otherwise, we can't get the location for last mapIndex.

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

No.

### How was this patch tested?

Updated existed test.

Closes #27850 from Ngone51/fix_getmaploction.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-09 15:53:34 +08:00
Kousuke Saruta 068bdd4415
[SPARK-31073][WEBUI] Add "shuffle write time" to task metrics summary in StagePage
### What changes were proposed in this pull request?

I've applied following changed to StagePage.
1. Added `Shuffle Write Time` to task metrics summary.
2. Added checkbox for `Shuffle Write Time` as an additional metrics.
3. Renamed `Write Time` column in task table to `Shuffle Write Time` and let it as an additional column.

### Why are the changes needed?

Task metrics summary doesn't show `Shuffle Write Time` even though it shows `Shuffle Read Blocked Time`.
`Shuffle Read Blocked Time` is let as an additional metrics so I also let `Shuffle Write Time` as an other additional metrics.

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

Yes. After this change, task metrics summary can show `Shuffle Write Time` and its visibility is controlled by a checkbox.
![additional-metrics-after](https://user-images.githubusercontent.com/4736016/76101844-677acb80-6012-11ea-9923-d95d852c775b.png)
![task-summary-after](https://user-images.githubusercontent.com/4736016/76101856-6ea1d980-6012-11ea-9670-3cf0ecd6faff.png)

`Write Time` column is already shown in task table but the title is ambiguous so I've renamed it as `Shuffle Write Time`.
After this change, this column is also additional column like `Shuffle Read Blocked Time`.
![tasks-table-after](https://user-images.githubusercontent.com/4736016/76102216-00a9e200-6013-11ea-9d51-1a6ce2abb0b9.png)

### How was this patch tested?

I've tested manually using following code and confirm the UI.
`sc.parallelize(1 to 1000).map(x => (x,x)).reduceByKey(_+_).collect`

Closes #27837 from sarutak/write-time.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-08 20:20:39 -07:00
beliefer bc490f383d [SPARK-31002][CORE][DOC] Add version information to the configuration of Core
### What changes were proposed in this pull request?
Add version information to the configuration of `Core`.
Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |  
SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath
SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions
SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath
spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |  
SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory
spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |  
spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |  
spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |  
spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |  
SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath
spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |  
spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |  
SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions
SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath
spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores
SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory
spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |  
spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |  
spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 |
spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |  
spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |
spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 |
spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |  
spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |  
spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |  
spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 |
spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |  
spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |  
spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27847 from beliefer/add-version-to-core-config-part-one.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-08 12:31:57 +09:00
beliefer e36227e2d9 [SPARK-30914][CORE][DOC] Add version information to the configuration of UI
### What changes were proposed in this pull request?
1.Add version information to the configuration of `UI`.
2.Update the docs of `UI`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |  
spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |  
spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |  
spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |  
spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |  
spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d |
spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d |
spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 |
spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |  
spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  
spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |  
spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |  
spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |  
spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  
spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27806 from beliefer/add-version-to-UI-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-06 11:08:57 +09:00
yi.wu 8d5ef2f766 [SPARK-31052][TEST][CORE] Fix flaky test "DAGSchedulerSuite.shuffle fetch failed on speculative task, but original task succeed"
### What changes were proposed in this pull request?

This PR fix the flaky test in #27050.

### Why are the changes needed?

`SparkListenerStageCompleted` is posted by `listenerBus` asynchronously. So, we should make sure listener has consumed the event before asserting completed stages.

See [error message](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119308/testReport/org.apache.spark.scheduler/DAGSchedulerSuite/shuffle_fetch_failed_on_speculative_task__but_original_task_succeed__SPARK_30388_/):

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: List(0, 1, 1) did not equal List(0, 1, 1, 0)
	at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:530)
	at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:529)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
	at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:503)
	at org.apache.spark.scheduler.DAGSchedulerSuite.$anonfun$new$88(DAGSchedulerSuite.scala:1976)
```

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

No.

### How was this patch tested?

Update test and test locally by no failure after running hundreds of times. Note, the failure is easy to reproduce when loop running the test for hundreds of times(e.g 200)

Closes #27809 from Ngone51/fix_flaky_spark_30388.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-03-05 10:56:49 -08:00
yi.wu 2257ce2443 [SPARK-31034][CORE] ShuffleBlockFetcherIterator should always create request for last block group
### What changes were proposed in this pull request?

This is a bug fix of #27280. This PR fix the bug where `ShuffleBlockFetcherIterator` may forget to create request for the last block group.

### Why are the changes needed?

When (all blocks).sum < `targetRemoteRequestSize` and (all blocks).length > `maxBlocksInFlightPerAddress` and (last block group).size < `maxBlocksInFlightPerAddress`,
`ShuffleBlockFetcherIterator` will not create a request for the last group. Thus, it will lost data for the reduce task.

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

No.

### How was this patch tested?

Updated test.

Closes #27786 from Ngone51/fix_no_request_bug.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 21:31:26 +08:00
Yuanjian Li 7db0af5785 [SPARK-30668][SQL][FOLLOWUP] Raise exception instead of silent change for new DateFormatter
### What changes were proposed in this pull request?
This is a follow-up work for #27441. For the cases of new TimestampFormatter return null while legacy formatter can return a value, we need to throw an exception instead of silent change. The legacy config will be referenced in the error message.

### Why are the changes needed?
Avoid silent result change for new behavior in 3.0.

### Does this PR introduce any user-facing change?
Yes, an exception is thrown when we detect legacy formatter can parse the string and the new formatter return null.

### How was this patch tested?
Extend existing UT.

Closes #27537 from xuanyuanking/SPARK-30668-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-05 15:29:39 +08:00
Kent Yao 3edab6cc1d [MINOR][CORE] Expose the alias -c flag of --conf for spark-submit
### What changes were proposed in this pull request?

-c is short for --conf, it was introduced since v1.1.0 but hidden from users until now

### Why are the changes needed?

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

no

expose hidden feature

### How was this patch tested?

Nah

Closes #27802 from yaooqinn/conf.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-03-04 20:37:51 -08:00
beliefer ebcff675e0 [SPARK-30889][SPARK-30913][CORE][DOC] Add version information to the configuration of Tests.scala and Worker
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Tests` and `Worker`.
2.Update the docs of `Worker`.

I sorted out some information of `Tests` show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.testing.memory | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-395d07dcd46359cca610ce74357f0bb4 |  
spark.testing.dynamicAllocation.scheduleInterval | 2.3.0 | SPARK-22864 | 4e9e6aee44bb2ddb41b567d659358b22fd824222#diff-b096353602813e47074ace09a3890d56 |  
spark.testing | 1.0.1 | SPARK-1606 | ce57624b8232159fe3ec6db228afc622133df591#diff-d239aee594001f8391676e1047a0381e |  
spark.test.noStageRetry | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-6a9ff7fb74fd490a50462d45db2d5e11 |  
spark.testing.reservedMemory | 1.6.0 | SPARK-12081 | 84c44b500b5c90dffbe1a6b0aa86f01699b09b96#diff-395d07dcd46359cca610ce74357f0bb4 |
spark.testing.nHosts | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.nExecutorsPerHost | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.nCoresPerExecutor | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.resources.warnings.testing | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  
spark.testing.resourceProfileManager | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |  

I sorted out some information of `Worker` show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.worker.resourcesFile | 3.0.0 | SPARK-27369 | 7cbe01e8efc3f6cd3a0cac4bcfadea8fcc74a955#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |  
spark.worker.timeout | 0.6.2 | None | e395aa295aeec6767df798bf1002b1f30983c1cd#diff-776a630ac2b2ec5fe85c07ca20a58fc0 |  
spark.worker.driverTerminateTimeout | 2.1.2 | SPARK-20843 | ebd72f453aa0b4f68760d28b3e93e6dd33856659#diff-829a8674171f92acd61007bedb1bfa4f |  
spark.worker.cleanup.enabled | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.cleanup.interval | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.cleanup.appDataTtl | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.preferConfiguredMasterAddress | 2.2.1 | SPARK-20529 | 75e5ea294c15ecfb7366ae15dce196aa92c87ca4#diff-916ca56b663f178f302c265b7ef38499 |  
spark.worker.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-48ca297b6536cb92362bec1487581f05 |  
spark.worker.ui.retainedExecutors | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 |
spark.worker.ui.retainedDrivers | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 |
spark.worker.ui.compressedLogFileLengthCacheSize | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-d239aee594001f8391676e1047a0381e |  
spark.worker.decommission.enabled | 3.1.0 | SPARK-20628 | d273a2bb0fac452a97f5670edd69d3e452e3e57e#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27783 from beliefer/add-version-to-tests-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 11:58:21 +09:00
yi.wu 87b93d32a6 [SPARK-31017][TEST][CORE] Test for shuffle requests packaging with different size and numBlocks limit
### What changes were proposed in this pull request?

Added 2 tests for `ShuffleBlockFetcherIteratorSuite`.

### Why are the changes needed?

When packaging shuffle fetch requests in `ShuffleBlockFetcherIterator`, there are two limitations: `maxBytesInFlight` and `maxBlocksInFlightPerAddress`. However, we don’t have test cases to test them both, e.g. the size limitation is hit before the numBlocks limitation.

We should add test cases in `ShuffleBlockFetcherIteratorSuite` to test:

1. the size limitation is hit before the numBlocks limitation
2. the numBlocks limitation is hit before the size limitation

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

No.

### How was this patch tested?

Added new tests.

Closes #27767 from Ngone51/add_test.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-04 20:21:48 +08:00
xuesenliang 7a4cf339d7 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks
### What changes were proposed in this pull request?

When a job finished, its running (re-submitted) map stages should be marked as finished if not used by other jobs. The running tasks of these stages are cancelled.

And the ListenerBus should be notified too, otherwise, these map stage items will stay on the "Active Stages" page of web UI and never gone.

For example:

Suppose job 0 has two stages: map stage 0 and result stage 1. Map stage 0 has two partitions, and its result stage 1 has two partitions too.

**Steps to reproduce the bug:**
1. map stage 0:    start task 0(```TID 0```) and task 1 (```TID 1```), then both finished successfully.
2. result stage 1:  start task 0(```TID 2```) and task 1 (```TID 3```)
3. result stage 1:  task 0(```TID 2```) finished successfully
4. result stage 1:  speculative task 1.1(```TID 4```) launched, but then failed due to FetchFailedException.
5. driver re-submits map stage 0 and result stage 1.
6. map stage 0 (retry 1): task0(```TID 5```) launched
7. result stage 1: task 1(```TID 3```) finished successfully, so job 0 finished.
8. map stage 0 is removed from ```runningStages``` and ```stageIdToStage```, because it doesn't belong to any job.
```
  private def DAGScheduler#cleanupStateForJobAndIndependentStages(job: ActiveJob): HashSet[Stage] = {
   ...
      stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach {
        case (stageId, stage) =>
            ...
            def removeStage(stageId: Int): Unit = {
              for (stage <- stageIdToStage.get(stageId)) {
                if (runningStages.contains(stage)) {
                  logDebug("Removing running stage %d".format(stageId))
                  runningStages -= stage
                }
                ...
              }
              stageIdToStage -= stageId
            }

            jobSet -= job.jobId
            if (jobSet.isEmpty) { // no other job needs this stage
              removeStage(stageId)
            }
          }
  ...
  }

```
9. map stage 0 (retry 1): task0(TID 5) finished successfully, but its stage 0 is not in ```stageIdToStage```, so the stage not ```markStageAsFinished```
```
  private[scheduler] def DAGScheduler#handleTaskCompletion(event: CompletionEvent): Unit = {
    val task = event.task
    val stageId = task.stageId
    ...
    if (!stageIdToStage.contains(task.stageId)) {
      postTaskEnd(event)
      // Skip all the actions if the stage has been cancelled.
      return
    }
    ...
```

#### Relevant spark driver logs as follows:

```
20/01/02 11:21:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 2 output partitions
20/01/02 11:21:45 INFO DAGScheduler: Final stage: ResultStage 1 (main at NativeMethodAccessorImpl.java:0)
20/01/02 11:21:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0)
20/01/02 11:21:45 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0)

20/01/02 11:21:45 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:21:45 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1))
20/01/02 11:21:45 INFO YarnClusterScheduler: Adding task set 0.0 with 2 tasks
20/01/02 11:21:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes)
20/01/02 11:21:45 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 9.76.13.26, executor 2, partition 1, PROCESS_LOCAL, 7705 bytes)
20/01/02 11:22:18 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 32491 ms on 9.179.143.4 (executor 1) (1/2)
20/01/02 11:22:26 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 40544 ms on 9.76.13.26 (executor 2) (2/2)
20/01/02 11:22:26 INFO DAGScheduler: ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 40.854 s
20/01/02 11:22:26 INFO YarnClusterScheduler: Removed TaskSet 0.0, whose tasks have all completed, from pool

20/01/02 11:22:26 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[6] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:22:26 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 1 (MapPartitionsRDD[6] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1))
20/01/02 11:22:26 INFO YarnClusterScheduler: Adding task set 1.0 with 2 tasks
20/01/02 11:22:26 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 2, 9.179.143.4, executor 1, partition 0, NODE_LOCAL, 7929 bytes)
20/01/02 11:22:26 INFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 3, 9.76.13.26, executor 2, partition 1, NODE_LOCAL, 7929 bytes)
20/01/02 11:22:26 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 2) in 79 ms on 9.179.143.4 (executor 1) (1/2)

20/01/02 11:22:26 INFO TaskSetManager: Marking task 1 in stage 1.0 (on 9.76.13.26) as speculatable because it ran more than 158 ms
20/01/02 11:22:26 INFO TaskSetManager: Starting task 1.1 in stage 1.0 (TID 4, 9.179.143.52, executor 3, partition 1, ANY, 7929 bytes)
20/01/02 11:22:26 WARN TaskSetManager: Lost task 1.1 in stage 1.0 (TID 4, 9.179.143.52, executor 3): FetchFailed(BlockManagerId(1, 9.179.143.4, 7337, None), shuffleId=0, mapId=0, reduceId=1, message=org.apache.spark.shuffle.FetchFailedException: Connection reset by peer)
20/01/02 11:22:26 INFO TaskSetManager: Task 1.1 in stage 1.0 (TID 4) failed, but the task will not be re-executed (either because the task failed with a shuffle data fetch failure, so the previous stage needs to be re-run, or because a different copy of the task has already succeeded).
20/01/02 11:22:26 INFO DAGScheduler: Marking ResultStage 1 (main at NativeMethodAccessorImpl.java:0) as failed due to a fetch failure from ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0)
20/01/02 11:22:26 INFO DAGScheduler: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) failed in 0.261 s due to org.apache.spark.shuffle.FetchFailedException: Connection reset by peer
20/01/02 11:22:26 INFO DAGScheduler: Resubmitting ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) and ResultStage 1 (main at NativeMethodAccessorImpl.java:0) due to fetch failure
20/01/02 11:22:26 INFO DAGScheduler: Resubmitting failed stages

20/01/02 11:22:26 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/01/02 11:22:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0))
20/01/02 11:22:26 INFO YarnClusterScheduler: Adding task set 0.1 with 1 tasks
20/01/02 11:22:26 INFO TaskSetManager: Starting task 0.0 in stage 0.1 (TID 5, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes)

// NOTE: Here should be "INFO TaskSetManager: Finished task 1.0 in stage 1.0 (TID 3) in 10000 ms on 9.76.13.26 (executor 2) (2/2)"
// and this bug is being fixed in https://issues.apache.org/jira/browse/SPARK-30404

20/01/02 11:22:36 INFO TaskSetManager: Ignoring task-finished event for 1.0 in stage 1.0 because task 1 has already completed successfully

20/01/02 11:22:36 INFO YarnClusterScheduler: Removed TaskSet 1.0, whose tasks have all completed, from pool
20/01/02 11:22:36 INFO DAGScheduler: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) finished in 10.131 s
20/01/02 11:22:36 INFO DAGScheduler: Job 0 finished: main at NativeMethodAccessorImpl.java:0, took 51.031212 s

20/01/02 11:22:58 INFO TaskSetManager: Finished task 0.0 in stage 0.1 (TID 5) in 32029 ms on 9.179.143.4 (executor 1) (1/1)
20/01/02 11:22:58 INFO YarnClusterScheduler: Removed TaskSet 0.1, whose tasks have all completed, from pool
```

### Why are the changes needed?

web UI is incorrect: ```stage 0 (retry 1)``` is finished, but it stays in ```Active Stages``` Page.

![active_stage](https://user-images.githubusercontent.com/4401756/71656718-71185680-2d77-11ea-8dbc-fd8085ab3dfb.png)

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

### How was this patch tested?
A new test case is added.

And test manually on cluster. The result is as follows:
![cancel_stage](https://user-images.githubusercontent.com/4401756/71658434-04a15580-2d7f-11ea-952b-dd8dd685f37d.png)

Closes #27050 from liangxs/master.

Authored-by: xuesenliang <xuesenliang@tencent.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-03 09:29:43 -06:00
yi.wu b517f991fe [SPARK-30969][CORE] Remove resource coordination support from Standalone
### What changes were proposed in this pull request?

Remove automatically resource coordination support from Standalone.

### Why are the changes needed?

Resource coordination is mainly designed for the scenario where multiple workers launched on the same host. However, it's, actually, a non-existed  scenario for today's Spark. Because, Spark now can start multiple executors in a single Worker, while it only allow one executor per Worker at very beginning. So, now, it really help nothing for user to launch multiple workers on the same host. Thus, it's not worth for us to bring over complicated implementation and potential high maintain cost for such an impossible scenario.

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

No, it's Spark 3.0 feature.

### How was this patch tested?

Pass Jenkins.

Closes #27722 from Ngone51/abandon_coordination.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-03-02 11:23:07 -08:00
Gengliang Wang 6b641430c3 [SPARK-30964][CORE][WEBUI] Accelerate InMemoryStore with a new index
### What changes were proposed in this pull request?

Spark uses the class `InMemoryStore` as the KV storage for live UI and history server(by default if no LevelDB file path is provided).
In `InMemoryStore`, all the task data in one application is stored in a hashmap, which key is the task ID and the value is the task data. This fine for getting or deleting with a provided task ID.
However, Spark stage UI always shows all the task data in one stage and the current implementation is to look up all the values in the hashmap. The time complexity is O(numOfTasks).
Also, when there are too many stages (>spark.ui.retainedStages), Spark will linearly try to look up all the task data of the stages to be deleted as well.

This can be very bad for a large application with many stages and tasks. We can improve it by allowing the natural key of an entity to have a real parent index. So that on each lookup with parent node provided, Spark can look up all the natural keys(in our case, the task IDs) first, and then find the data with the natural keys in the hashmap.

### Why are the changes needed?

The in-memory KV store becomes really slow for large applications. We can improve it with a new index. The performance can be 10 times, 100 times, even 1000 times faster.
This is also possible to make the Spark driver more stable for large applications.

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

No

### How was this patch tested?

Existing unit tests.
Also, I run a benchmark with the following code
```
  val store = new InMemoryStore()
  val numberOfTasksPerStage = 10000
   (0 until 1000).map { sId =>
     (0 until numberOfTasksPerStage).map { taskId =>
       val task = newTaskData(sId * numberOfTasksPerStage + taskId, "SUCCESS", sId)
       store.write(task)
     }
   }
  val appStatusStore = new AppStatusStore(store)
  var start = System.nanoTime()
  appStatusStore.taskSummary(2, attemptId, Array(0, 0.25, 0.5, 0.75, 1))
  println("task summary run time: " + ((System.nanoTime() - start) / 1000000))
  val stageIds = Seq(1, 11, 66, 88)
  val stageKeys = stageIds.map(Array(_, attemptId))
  start = System.nanoTime()
  store.removeAllByIndexValues(classOf[TaskDataWrapper], TaskIndexNames.STAGE,
    stageKeys.asJavaCollection)
   println("clean up tasks run time: " + ((System.nanoTime() - start) / 1000000))
```

Task summary before the changes: 98642ms
Task summary after the changes: 120ms

Task clean up before the changes:  4900ms
Task clean up before the changes: 4ms

It's 800x faster after the changes in the micro-benchmark.

Closes #27716 from gengliangwang/liveUIStore.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-02 15:48:48 +08:00
beliefer 71365c2502 [SPARK-30912][CORE][DOC] Add version information to the configuration of Streaming.scala
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Streaming`.
2.Update the docs of `Streaming`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.streaming.dynamicAllocation.enabled | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.testing | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.minExecutors | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.maxExecutors | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingInterval | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingUpRatio | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |
spark.streaming.dynamicAllocation.scalingDownRatio | 3.0.0 | SPARK-26941 | cad475dcc9376557f882859856286e858002389a#diff-335c3bbf4ca27cf65a6f850b1a7c89dd |

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27745 from beliefer/add-version-to-streaming-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:16:40 +09:00
beliefer c63366a693 [SPARK-30891][CORE][DOC] Add version information to the configuration of History
### What changes were proposed in this pull request?
1.Add version information to the configuration of `History`.
2.Update the docs of `History`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.history.fs.logDirectory | 1.1.0 | SPARK-1768 | 21ddd7d1e9f8e2a726427f32422c31706a20ba3f#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.safemodeCheck.interval | 1.6.0 | SPARK-11020 | cf04fdfe71abc395163a625cc1f99ec5e54cc07e#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.update.interval | 1.4.0 | SPARK-6046 | 4527761bcd6501c362baf2780905a0018b9a74ba#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.cleaner.enabled | 1.3.0 | SPARK-3562 | 8942b522d8a3269a2a357e3a274ed4b3e66ebdde#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e | Branch branch-1.3 does not exist, exists in branch-1.4, but it is 1.3.0-SNAPSHOT in pom.xml
spark.history.fs.cleaner.interval | 1.4.0 | SPARK-5933 | 1991337336596f94698e79c2366f065c374128ab#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |
spark.history.fs.cleaner.maxAge | 1.4.0 | SPARK-5933 | 1991337336596f94698e79c2366f065c374128ab#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |
spark.history.fs.cleaner.maxNum | 3.0.0 | SPARK-28294 | bbc2be4f425c4c26450e1bf21db407e81046ce21#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.store.path | 2.3.0 | SPARK-20642 | 74daf622de4e534d5a5929b424a6e836850eefad#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.store.maxDiskUsage | 2.3.0 | SPARK-20654 | 8b497046c647a21bbed1bdfbdcb176745a1d5cd5#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.ui.port | 1.0.0 | SPARK-1276 | 9ae80bf9bd3e4da7443af97b41fe26aa5d35d70b#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.fs.inProgressOptimization.enabled | 2.4.0 | SPARK-6951 | 653fe02415a537299e15f92b56045569864b6183#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.fs.endEventReparseChunkSize | 2.4.0 | SPARK-6951 | 653fe02415a537299e15f92b56045569864b6183#diff-19f35f981fdc5b0a46f070b879a9a9fc |  
spark.history.fs.eventLog.rolling.maxFilesToRetain | 3.0.0 | SPARK-30481 | a2fe73b83c0e7c61d1c83b236565a71e3d005a71#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.eventLog.rolling.compaction.score.threshold | 3.0.0 | SPARK-30481 | a2fe73b83c0e7c61d1c83b236565a71e3d005a71#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.interval | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.fs.driverlog.cleaner.maxAge | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.ui.acls.enable | 1.0.1 | Spark 1489 | c8dd13221215275948b1a6913192d40e0c8cbadd#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.ui.admin.acls | 2.1.1 | SPARK-19033 | 4ca1788805e4a0131ba8f0ccb7499ee0e0242837#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.ui.admin.acls.groups | 2.1.1 | SPARK-19033 | 4ca1788805e4a0131ba8f0ccb7499ee0e0242837#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.fs.numReplayThreads | 2.0.0 | SPARK-13988 | 6fdd0e32a6c3fdce1f3f7e1f8d252af05c419f7b#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.retainedApplications | 1.0.0 | SPARK-1276 | 9ae80bf9bd3e4da7443af97b41fe26aa5d35d70b#diff-b49b5b9c31ddb36a9061004b5b723058 |
spark.history.provider | 1.1.0 | SPARK-1768 | 21ddd7d1e9f8e2a726427f32422c31706a20ba3f#diff-a7befb99e7bd7e3ab5c46c2568aa5b3e |  
spark.history.kerberos.enabled | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.kerberos.principal | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.kerberos.keytab | 1.0.1 | Spark-1490 | 866b03ef4d27b2160563b58d577de29ba6eb4442#diff-b49b5b9c31ddb36a9061004b5b723058 |  
spark.history.custom.executor.log.url | 3.0.0 | SPARK-26311 | ae5b2a6a92be4986ef5b8062d7fb59318cff6430#diff-6bddeb5e25239974fc13db66266b167b |  
spark.history.custom.executor.log.url.applyIncompleteApplication | 3.0.0 | SPARK-26311 | ae5b2a6a92be4986ef5b8062d7fb59318cff6430#diff-6bddeb5e25239974fc13db66266b167b |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27751 from beliefer/add-version-to-history-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:15:49 +09:00
beliefer 3beb4f875d [SPARK-30908][CORE][DOC] Add version information to the configuration of Kryo
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Kryo`.
2.Update the docs of `Kryo`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.kryo.registrationRequired | 1.1.0 | SPARK-2102 | efdaeb111917dd0314f1d00ee8524bed1e2e21ca#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryo.registrator | 0.5.0 | None | 91c07a33d90ab0357e8713507134ecef5c14e28a#diff-792ed56b3398163fa14e8578549d0d98 | This is not a release version, do we need to record it?
spark.kryo.classesToRegister | 1.2.0 | SPARK-1813 | 6bb56faea8d238ea22c2de33db93b1b39f492b3a#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.kryo.unsafe | 2.1.0 | SPARK-928 | bc167a2a53f5a795d089e8a884569b1b3e2cd439#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryo.pool | 3.0.0 | SPARK-26466 | 38f030725c561979ca98b2a6cc7ca6c02a1f80ed#diff-a3c6b992784f9abeb9f3047d3dcf3ed9 |  
spark.kryo.referenceTracking | 0.8.0 | None | 0a8cc309211c62f8824d76618705c817edcf2424#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryoserializer.buffer | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |  
spark.kryoserializer.buffer.max | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27734 from beliefer/add-version-to-kryo-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 15:14:47 +09:00
Thomas Graves 0e2ca11d80 [SPARK-29149][YARN] Update YARN cluster manager For Stage Level Scheduling
### What changes were proposed in this pull request?

Yarn side changes for Stage level scheduling.  The previous PR for dynamic allocation changes was https://github.com/apache/spark/pull/27313

Modified the data structures to store things on a per ResourceProfile basis.
 I tried to keep the code changes to a minimum, the main loop that requests just goes through each Resourceprofile and the logic inside for each one stayed very close to the same.
On submission we now have to give each ResourceProfile a separate yarn Priority because yarn doesn't support asking for containers with different resources at the same Priority. We just use the profile id as the priority level.
Using a different Priority actually makes things easier when the containers come back to match them again which ResourceProfile they were requested for.
The expectation is that yarn will only give you a container with resource amounts you requested or more. It should never give you a container if it doesn't satisfy your resource requests.

If you want to see the full feature changes you can look at https://github.com/apache/spark/pull/27053/files for reference

### Why are the changes needed?

For stage level scheduling YARN support.

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

no

### How was this patch tested?

Tested manually on YARN cluster and then unit tests.

Closes #27583 from tgravescs/SPARK-29149.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-28 15:23:33 -06:00
Thomas Graves 6c0c41fa0d [SPARK-30987][CORE] Increase the timeout on local-cluster waitUntilExecutorsUp calls
### What changes were proposed in this pull request?

The ResourceDiscoveryPlugin tests intermittently timeout. They are timing out on just bringing up the local-cluster. I am not able to reproduce locally.  I suspect the jenkins boxes are overloaded and taking longer then 10 seconds. There was another jira SPARK-29139 that increased timeout for some other of these as well. So try increasing the timeout to 60 seconds.

Examples of timeouts:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119030/testReport/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119005/testReport/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119029/testReport/

### Why are the changes needed?

tests should no longer intermittently fail.

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

no
### How was this patch tested?

unit tests ran.

Closes #27738 from tgravescs/SPARK-30987.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-28 11:43:05 -08:00
yi.wu 3b69796a89 [SPARK-30947][CORE] Log better message when accelerate resource is empty
### What changes were proposed in this pull request?

Try to log better message when accelerate resource is empty.

### Why are the changes needed?

Otherwise, it's weird to see cpu/memory resources after logging **that** resources is empty:

```
20/02/25 21:47:55 INFO ResourceUtils: ==============================================================
20/02/25 21:47:55 INFO ResourceUtils: Resources for spark.driver:

20/02/25 21:47:55 INFO ResourceUtils: ==============================================================
20/02/25 21:47:55 INFO SparkContext: Submitted application: Spark shell
20/02/25 21:47:55 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0)
20/02/25 21:47:55 INFO ResourceProfile: Limiting resource is  at -1 tasks per executor
```

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

NO.

### How was this patch tested?

Tested manually.

Closes #27693 from Ngone51/dont_log_resource.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-28 10:54:38 -08:00
Kent Yao 1383bd459a [SPARK-30970][K8S][CORE] Fix NPE while resolving k8s master url
### What changes were proposed in this pull request?

```
bin/spark-sql --master  k8s:///https://kubernetes.docker.internal:6443 --conf spark.kubernetes.container.image=yaooqinn/spark:v2.4.4
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.util.Utils$.checkAndGetK8sMasterUrl(Utils.scala:2739)
	at org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:261)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:774)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:161)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:184)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:920)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:929)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
```
Althrough `k8s:///https://kubernetes.docker.internal:6443` is a wrong master url but should not throw npe
The `case null` will never be touched.
3f4060c340/core/src/main/scala/org/apache/spark/util/Utils.scala (L2772-L2776)

### Why are the changes needed?

bug fix

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

no

### How was this patch tested?

add ut case

Closes #27721 from yaooqinn/SPARK-30970.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-28 00:01:20 -08:00
beliefer 325bf56e73 [SPARK-30888][CORE][DOC] Add version information to the configuration of Network
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Network`.
2.Update the docs of `Network`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |  
spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |  
spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |  
spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |  
spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |  
spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |  
spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |  
spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  
spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27674 from beliefer/add-version-to-network-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 11:05:11 +09:00
beliefer c2857501d5 [SPARK-30909][CORE][DOC] Add version information to the configuration of Python
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Python`.
2.Update the docs of `Python`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.python.worker.reuse | 1.2.0 | SPARK-3030 | 2aea0da84c58a179917311290083456dfa043db7#diff-0a67bc4d171abe4df8eb305b0f4123a2 |  
spark.python.task.killTimeout | 2.2.2 | SPARK-22535 | be68f86e11d64209d9e325ce807025318f383bea#diff-0a67bc4d171abe4df8eb305b0f4123a2 |  
spark.python.use.daemon | 2.3.0 | SPARK-22554 | 57c5514de9dba1c14e296f85fb13fef23ce8c73f#diff-9008ad45db34a7eee2e265a50626841b |  
spark.python.daemon.module | 2.4.0 | SPARK-22959 | afae8f2bc82597593595af68d1aa2d802210ea8b#diff-9008ad45db34a7eee2e265a50626841b |  
spark.python.worker.module | 2.4.0 | SPARK-22959 | afae8f2bc82597593595af68d1aa2d802210ea8b#diff-9008ad45db34a7eee2e265a50626841b |  
spark.executor.pyspark.memory | 2.4.0 | SPARK-25004 | 7ad18ee9f26e75dbe038c6034700f9cd4c0e2baa#diff-6bdad48cfc34314e89599655442ff210 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27704 from beliefer/add-version-to-python-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 10:57:34 +09:00
beliefer 776e21af40 [SPARK-30910][CORE][DOC] Add version information to the configuration of R
### What changes were proposed in this pull request?
1.Add version information to the configuration of `R`.
2.Update the docs of `R`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.r.backendConnectionTimeout | 2.1.0 | SPARK-17919 | 2881a2d1d1a650a91df2c6a01275eba14a43b42a#diff-025470e1b7094d7cf4a78ea353fb3981 |  
spark.r.numRBackendThreads | 1.4.0 | SPARK-8282 | 28e8a6ea65fd08ab9cefc4d179d5c66ffefd3eb4#diff-697f7f2fc89808e0113efc71ed235db2 |  
spark.r.heartBeatInterval | 2.1.0 | SPARK-17919 | 2881a2d1d1a650a91df2c6a01275eba14a43b42a#diff-fe903bf14db371aa320b7cc516f2463c |  
spark.sparkr.r.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |  
spark.r.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |  

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27708 from beliefer/add-version-to-R-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 10:56:38 +09:00
gatorsmile 28b8713036 [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT
### What changes were proposed in this pull request?
This patch is to bump the master branch version to 3.1.0-SNAPSHOT.

### Why are the changes needed?
N/A

### Does this PR introduce any user-facing change?
N/A

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

Closes #27698 from gatorsmile/updateVersion.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-25 19:44:31 -08:00
yi.wu e9fd52282e [SPARK-30689][CORE][FOLLOW-UP] Rename config name of discovery plugin
### What changes were proposed in this pull request?

Rename config `spark.resources.discovery.plugin` to `spark.resources.discoveryPlugin`.

Also, as a side minor change: labeled `ResourceDiscoveryScriptPlugin` as `DeveloperApi` since it's not for end user.

### Why are the changes needed?

Discovery plugin doesn't need to reserve the "discovery" namespace here and it's more consistent with the interface name `ResourceDiscoveryPlugin` if we use `discoveryPlugin` instead.

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

No, it's newly added in Spark3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27689 from Ngone51/spark_30689_followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 11:55:05 +09:00
Jungtaek Lim (HeartSaVioR) 9ea6c0a897
[SPARK-30943][SS] Show "batch ID" in tool tip string for Structured Streaming UI graphs
### What changes were proposed in this pull request?

This patch changes the tool tip string in Structured Streaming UI graphs to show batch ID (and timestamp as well) instead of only showing timestamp, which was a key for DStream but no longer a key for Structured Streaming.

This patch does some refactoring as there're some spots on confusion between js file for streaming and structured streaming.

Note that this patch doesn't actually change the x axis, as once we change it we should decouple the logic for graphs between streaming and structured streaming. It won't change UX meaningfully as in x axis we only show min and max which we still would like to know about "time" as well as batch ID.

### Why are the changes needed?

In Structured Streaming, everything is aligned for "batch ID" where the UI is only showing timestamp - end users have to manually find and correlate batch ID and the timestamp which is clearly a huge pain.

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

No

### How was this patch tested?

Manually tested. Screenshots:

![Screen Shot 2020-02-25 at 7 22 38 AM](https://user-images.githubusercontent.com/1317309/75197701-40b2ce80-57a2-11ea-9578-c2eb2d1091de.png)
![Screen Shot 2020-02-25 at 7 22 44 AM](https://user-images.githubusercontent.com/1317309/75197704-427c9200-57a2-11ea-9439-e0a8303d0860.png)
![Screen Shot 2020-02-25 at 7 22 58 AM](https://user-images.githubusercontent.com/1317309/75197706-43152880-57a2-11ea-9617-1276c3ba181e.png)
![Screen Shot 2020-02-25 at 7 23 04 AM](https://user-images.githubusercontent.com/1317309/75197708-43152880-57a2-11ea-9de2-7d37eaf88102.png)
![Screen Shot 2020-02-25 at 7 23 31 AM](https://user-images.githubusercontent.com/1317309/75197710-43adbf00-57a2-11ea-9ae4-4e292de39c36.png)

Closes #27687 from HeartSaVioR/SPARK-30943.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-02-25 15:29:36 -08:00
Thomas Graves c46c067f39 [SPARK-30942] Fix the warning for requiring cores to be limiting resources
### What changes were proposed in this pull request?

fix the warning for limiting resources when we don't know the number of executor cores. The issue is that there are places in the Spark code that use cores/task cpus to calculate slots and until the entire Stage level scheduling feature is in, we have to rely on the cores being the limiting resource.

Change the check to only warn when custom resources are specified.

### Why are the changes needed?

fix the check and warn when we should

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

A warning is printed

### How was this patch tested?

manually tested spark-shell with standalone mode, yarn, local mode.

Closes #27686 from tgravescs/SPARK-30942.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-25 10:55:56 -06:00
Shixiong Zhu 3126557b07 [SPARK-30936][CORE] Set FAIL_ON_UNKNOWN_PROPERTIES to false by default to parse Spark events
### What changes were proposed in this pull request?

Set `FAIL_ON_UNKNOWN_PROPERTIES` to `false` in `JsonProtocol` to allow ignore unknown fields in a Spark event. After this change, if we add new fields to a Spark event parsed by `ObjectMapper`, the event json string generated by a new Spark version can still be read by an old Spark History Server.

Since Spark History Server is an extra service, it usually takes time to upgrade, and it's possible that a Spark application is upgraded before SHS. Forwards-compatibility will allow an old SHS to support new Spark applications (may lose some new features but most of functions should still work).

### Why are the changes needed?

`JsonProtocol` is supposed to provide strong backwards-compatibility and forwards-compatibility guarantees: any version of Spark should be able to read JSON output written by any other version, including newer versions.

However, the forwards-compatibility guarantee is broken for events parsed by `ObjectMapper`. If a new field is added to an event parsed by `ObjectMapper` (e.g., 6dc5921e66 (diff-dc5c7a41fbb7479cef48b67eb41ad254R33)), the event json string generated by a new Spark version cannot be parsed by an old version of SHS right now.

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

No.

### How was this patch tested?

The new added tests.

Closes #27680 from zsxwing/SPARK-30936.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-25 12:28:31 +08:00
beliefer 7911de9d10 [SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy
### What changes were proposed in this pull request?
1.Add version information to the configuration of `Deploy`.
2.Update the docs of `Deploy`.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 |		deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT
spark.deploy.recoveryDirectory | 0.8.1 | None |			d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.zookeeper.url | 0.8.1 | None |			d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c |
spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 |
spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 |
spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 |
spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 |

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27668 from beliefer/add-version-to-deploy-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-25 11:39:11 +09:00
beliefer 59d6d5cbb0 [SPARK-30840][CORE][SQL] Add version property for ConfigEntry and ConfigBuilder
### What changes were proposed in this pull request?
Spark `ConfigEntry` and `ConfigBuilder` missing Spark version information of each configuration at release. This is not good for Spark user when they visiting the page of spark configuration.
http://spark.apache.org/docs/latest/configuration.html
The new Spark SQL config docs looks like:
![sql配置截屏](https://user-images.githubusercontent.com/8486025/74604522-cb882f00-50f9-11ea-8683-57a90f9e3347.png)

```
> SET -v
spark.sql.adaptive.enabled      false   When true, enable adaptive query execution.
spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin       0.2     The relation with a non-empty partition ratio lower than this config will not be considered as the build side of a broadcast-hash join in adaptive execution regardless of its size.This configuration only has an effect when 'spark.sql.adaptive.enabled' is enabled.
spark.sql.adaptive.optimizeSkewedJoin.enabled   true    When true and adaptive execution is enabled, a skewed join is automatically handled at runtime.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor     10      A partition is considered as a skewed partition if its size is larger than this factor multiple the median partition size and also larger than  spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionMaxSplits  5       Configures the maximum number of task to handle a skewed partition in adaptive skewedjoin.
spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold      64MB    Configures the minimum size in bytes for a partition that is considered as a skewed partition in adaptive skewed join.
spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled    true    Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks one by one, fetching continuous shuffle blocks for the same map task in batch can reduce IO and improve performance. Note, multiple continuous blocks exist in single fetch request only happen when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled, this feature also depends on a relocatable serializer, the concatenation support codec in use and the new version shuffle fetch protocol.
spark.sql.adaptive.shuffle.localShuffleReader.enabled   true    When true and 'spark.sql.adaptive.enabled' is enabled, this enables the optimization of converting the shuffle reader to local shuffle reader for the shuffle exchange of the broadcast hash join in probe side.
spark.sql.adaptive.shuffle.maxNumPostShufflePartitions  <undefined>     The advisory maximum number of post-shuffle partitions used in adaptive execution. This is used as the initial number of pre-shuffle partitions. By default it equals to spark.sql.shuffle.partitions. This configuration only has an effect when 'spark.sql.adaptive.enabled' and 'spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled' is enabled.
```

**Note**: Because there are so many configuration items that are exposed and require a lot of finishing, I will add the version numbers of these configuration items in another PR.

### Why are the changes needed?
Supplemental configuration version information.

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

### How was this patch tested?
Exists UT

Closes #27592 from beliefer/add-version-to-config.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-22 09:46:42 +09:00
sarthfrey-db 274b328f57 [SPARK-30667][CORE] Add all gather method to BarrierTaskContext
Fix for #27395

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

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

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

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27640 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-02-21 11:40:28 -08:00
Yuanjian Li a5efbb284e [SPARK-30809][SQL] Review and fix issues in SQL API docs
### What changes were proposed in this pull request?
- Add missing `since` annotation.
- Don't show classes under `org.apache.spark.sql.dynamicpruning` package in API docs.
- Fix the scope of `xxxExactNumeric` to remove it from the API docs.

### Why are the changes needed?
Avoid leaking APIs unintentionally in Spark 3.0.0.

### Does this PR introduce any user-facing change?
No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?
Manually generated the API docs and verified the above issues have been fixed.

Closes #27560 from xuanyuanking/SPARK-30809.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-21 17:03:22 +08:00
Dongjoon Hyun fc4e56a54c [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9
This PR aims to upgrade Py4J to `0.10.9` for better Python 3.7 support in Apache Spark 3.0.0 (master/branch-3.0). This is not for `branch-2.4`.

- Apache Spark 3.0.0 is using `Py4J 0.10.8.1` (released on 2018-10-21) because `0.10.8.1` was the first official release to support Python 3.7.
    - https://www.py4j.org/changelog.html#py4j-0-10-8-and-py4j-0-10-8-1
- `Py4J 0.10.9` was released on January 25th 2020 with better Python 3.7 support and `magic_member` bug fix.
    - https://github.com/bartdag/py4j/releases/tag/0.10.9
    - https://www.py4j.org/changelog.html#py4j-0-10-9

No.

Pass the Jenkins with the existing tests.

Closes #27641 from dongjoon-hyun/SPARK-30884.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-20 09:09:30 -08:00
HyukjinKwon 7c4ad6316e [SPARK-29148][CORE][FOLLOW-UP] Don't dynamic allocation warning when it's disabled
### What changes were proposed in this pull request?

Currently, after https://github.com/apache/spark/pull/27313, it shows the warning about dynamic allocation which is disabled by default.

```bash
$ ./bin/spark-shell
```

```
...
20/02/18 11:04:56 WARN ResourceProfile: Please ensure that the number of slots available on your executors is
limited by the number of cores to task cpus and not another custom resource. If cores is not the limiting resource
then dynamic allocation will not work properly!
```

This PR brings back the configuration checking for this warning. Seems mistakenly removed at https://github.com/apache/spark/pull/27313/files#diff-364713d7776956cb8b0a771e9b62f82dL2841

### Why are the changes needed?

To remove false warning.

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

Yes, it will don't show the warning. It's master only change so no user-facing to end users.

### How was this patch tested?

Manually tested.

Closes #27615 from HyukjinKwon/SPARK-29148.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-19 23:01:49 -08:00
Xingbo Jiang e32411eb07 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit af63971cb7.
2020-02-19 17:04:47 -08:00
sarthfrey-db af63971cb7 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

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

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
(cherry picked from commit 57254c9719)
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-19 12:10:51 -08:00
Prakhar Jain e086951349 [SPARK-30786][CORE] Fix Block replication failure propogation issue in BlockManager
### What changes were proposed in this pull request?
Currently the uploadBlockSync api in BlockTransferService always succeeds irrespective of whether the BlockManager was able to successfully replicate a block on peer block manager or not. This PR makes sure that the NettyBlockRpcServer invokes onFailure callback when it is not able to replicate the block to itself because of any reason. The onFailure callback makes sure that the BlockTransferService on client side gets the failure and retry replication the Block on some other BlockManager.

### Why are the changes needed?
Currently the Spark Block replication retry logic is not working correctly. It doesn't retry on other Block managers even when replication fails on 1 of the peers.

A user can cache an DataFrame with different replication factor. Ex - df.persist(StorageLevel.MEMORY_ONLY_2) - This will cache each partition at two different BlockManagers. When a DataFrame partition is computed first time, it is firstly stored locally on the local BlockManager and then it is replicated to other block managers based on replication factor config. The replication of block to other block managers might fail because of memory/network etc issues and so there is already provision to retry the replication on some other peer based on "spark.storage.maxReplicationFailures" config, Currently when this replication fails, the client does not know about the failure and so it doesn't retry on other peers. This PR fixes this issue.

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

### How was this patch tested?
Added Unit Test.

Closes #27539 from prakharjain09/bm_replicate.

Authored-by: Prakhar Jain <prakharjain09@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-19 20:23:22 +08:00
yi.wu 68d7edf949 [SPARK-30812][SQL][CORE] Revise boolean config name to comply with new config naming policy
### What changes were proposed in this pull request?

Revise below config names to comply with [new config naming policy](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html):

SQL:
* spark.sql.execution.subquery.reuse.enabled / [SPARK-27083](https://issues.apache.org/jira/browse/SPARK-27083)
* spark.sql.legacy.allowNegativeScaleOfDecimal.enabled / [SPARK-30252](https://issues.apache.org/jira/browse/SPARK-30252)
* spark.sql.adaptive.optimizeSkewedJoin.enabled / [SPARK-29544](https://issues.apache.org/jira/browse/SPARK-29544)
* spark.sql.legacy.property.nonReserved / [SPARK-30183](https://issues.apache.org/jira/browse/SPARK-30183)
* spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled / [SPARK-26389](https://issues.apache.org/jira/browse/SPARK-26389)
* spark.sql.analyzer.failAmbiguousSelfJoin.enabled / [SPARK-28344](https://issues.apache.org/jira/browse/SPARK-28344)
* spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled / [SPARK-30074](https://issues.apache.org/jira/browse/SPARK-30074)
* spark.sql.execution.pandas.arrowSafeTypeConversion / [SPARK-25811](https://issues.apache.org/jira/browse/SPARK-25811)
* spark.sql.legacy.looseUpcast / [SPARK-24586](https://issues.apache.org/jira/browse/SPARK-24586)
* spark.sql.legacy.arrayExistsFollowsThreeValuedLogic / [SPARK-28052](https://issues.apache.org/jira/browse/SPARK-28052)
* spark.sql.sources.ignoreDataLocality.enabled / [SPARK-29189](https://issues.apache.org/jira/browse/SPARK-29189)
* spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled / [SPARK-9853](https://issues.apache.org/jira/browse/SPARK-9853)

CORE:
* spark.eventLog.erasureCoding.enabled / [SPARK-25855](https://issues.apache.org/jira/browse/SPARK-25855)
* spark.shuffle.readHostLocalDisk.enabled / [SPARK-30235](https://issues.apache.org/jira/browse/SPARK-30235)
* spark.scheduler.listenerbus.logSlowEvent.enabled / [SPARK-29001](https://issues.apache.org/jira/browse/SPARK-29001)
* spark.resources.coordinate.enable / [SPARK-27371](https://issues.apache.org/jira/browse/SPARK-27371)
* spark.eventLog.logStageExecutorMetrics.enabled / [SPARK-23429](https://issues.apache.org/jira/browse/SPARK-23429)

### Why are the changes needed?

To comply with the config naming policy.

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

No. Configurations listed above are all newly added in Spark 3.0.

### How was this patch tested?

Pass Jenkins.

Closes #27563 from Ngone51/revise_boolean_conf_name.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 20:39:50 +08:00
Ajith 2854091d12 [SPARK-22590][SQL] Copy sparkContext.localproperties to child thread in BroadcastExchangeExec.executionContext
### What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the broadcast execution thread in `org.apache.spark.sql.execution.exchange.BroadcastExchangeExec#executionContext`

### Why are the changes needed?
When executing `BroadcastExchangeExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads.
These threads are created in the executionContext (thread pools). Each Thread pool has a default `keepAliveSeconds` of 60 seconds for idle threads.
Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob`

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

### How was this patch tested?
Added UT

Closes #27266 from ajithme/broadcastlocalprop.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 02:26:52 +08:00
Liupengcheng 5b873420b0 [SPARK-30346][CORE] Improve logging when events dropped
### What changes were proposed in this pull request?

Make logging events dropping every 60s works fine, the orignal implementaion some times not working due to susequent events comming and updating the DroppedEventCounter

### Why are the changes needed?

Currenly, the logging may be skipped and delayed a long time under high concurrency, that make debugging hard. So This PR will try to fix it.

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

No

### How was this patch tested?

NA

Closes #27002 from liupc/Improve-logging-dropped-events-and-logging-threadDump.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 20:16:31 +08:00
Jungtaek Lim (HeartSaVioR) 5445fe9288 [SPARK-30827][DOCS] Document direct relationship among configurations in "spark.history.*" namespace
### What changes were proposed in this pull request?

This patch adds direct relationship among configurations under "spark.history" namespace.

### Why are the changes needed?

Refer the discussion thread: https://lists.apache.org/thread.html/r43c4e57cace116aca1f0f099e8a577cf202859e3671a04077867b84a%40%3Cdev.spark.apache.org%3E

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

No.

### How was this patch tested?

Locally ran jekyll and confirmed. Screenshots for the modified spots:

<img width="1159" alt="Screen Shot 2020-02-15 at 8 20 14 PM" src="https://user-images.githubusercontent.com/1317309/74587003-d5922b00-5030-11ea-954b-ee37fc08470a.png">
<img width="1158" alt="Screen Shot 2020-02-15 at 8 20 44 PM" src="https://user-images.githubusercontent.com/1317309/74587005-d62ac180-5030-11ea-98fc-98b1c9d83ff4.png">
<img width="1149" alt="Screen Shot 2020-02-15 at 8 19 56 PM" src="https://user-images.githubusercontent.com/1317309/74587002-d1660d80-5030-11ea-84b5-dec3d7f5c97c.png">

Closes #27575 from HeartSaVioR/SPARK-30827.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-17 20:45:24 +09:00
Jungtaek Lim (HeartSaVioR) 446b2d2653 [SPARK-28869][DOCS][FOLLOWUP] Add direct relationship between configs for rolling event log
### What changes were proposed in this pull request?

This patch addresses the post-hoc review comment linked here - https://github.com/apache/spark/pull/25670#discussion_r373304076

### Why are the changes needed?

We would like to explicitly document the direct relationship before we finish up structuring of configurations.

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

No.

### How was this patch tested?

N/A

Closes #27576 from HeartSaVioR/SPARK-28869-FOLLOWUP-doc.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-17 20:41:56 +09:00
Wenchen Fan 619274ed36 [DOC] add config naming guideline
### What changes were proposed in this pull request?

Add docs to describe the config naming guideline.

### Why are the changes needed?

To encourage contributors to name configs more consistently.

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

no

### How was this patch tested?

N/A

Closes #27577 from cloud-fan/config.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-17 12:40:52 +08:00
Holden Karau d273a2bb0f [SPARK-20628][CORE][K8S] Start to improve Spark decommissioning & preemption support
This PR is based on an existing/previou PR - https://github.com/apache/spark/pull/19045

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

This changes adds a decommissioning state that we can enter when the cloud provider/scheduler lets us know we aren't going to be removed immediately but instead will be removed soon. This concept fits nicely in K8s and also with spot-instances on AWS / preemptible instances all of which we can get a notice that our host is going away. For now we simply stop scheduling jobs, in the future we could perform some kind of migration of data during scale-down, or at least stop accepting new blocks to cache.

There is a design document at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE/edit?usp=sharing

### Why are the changes needed?

With more move to preemptible multi-tenancy, serverless environments, and spot-instances better handling of node scale down is required.

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

There is no API change, however an additional configuration flag is added to enable/disable this behaviour.

### How was this patch tested?

New integration tests in the Spark K8s integration testing. Extension of the AppClientSuite to test decommissioning seperate from the K8s.

Closes #26440 from holdenk/SPARK-20628-keep-track-of-nodes-which-are-going-to-be-shutdown-r4.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2020-02-14 12:36:52 -08:00
Xingbo Jiang fa3517cdb1 Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext"
This reverts commit 57254c9719.
2020-02-13 17:43:55 -08:00
sarthfrey-db 57254c9719 [SPARK-30667][CORE] Add allGather method to BarrierTaskContext
### What changes were proposed in this pull request?

The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call.

### Why are the changes needed?

There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on.

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

Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs.

### How was this patch tested?

Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID.

An example through the Python API:
```python
>>> from pyspark import BarrierTaskContext
>>>
>>> def f(iterator):
...     context = BarrierTaskContext.get()
...     return [context.allGather('{}'.format(context.partitionId()))]
...
>>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0]
[u'3', u'1', u'0', u'2']
```

Closes #27395 from sarthfrey/master.

Lead-authored-by: sarthfrey-db <sarth.frey@databricks.com>
Co-authored-by: sarthfrey <sarth.frey@gmail.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2020-02-13 16:15:00 -08:00
Thomas Graves 496f6ac860 [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes
### What changes were proposed in this pull request?

This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile.  Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile.

The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles.

I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents.

There are various places in the code that use executor "slots" for things.  The ResourceProfile adds functionality to keep that calculation in it.   This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there.
This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things.

This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier.   That full set of changes will come with the scheduler PR that will be after this one.

The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue.

The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well.

### Why are the changes needed?

Needed for stage level scheduling feature.

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

No user facing api changes added here.

### How was this patch tested?

Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases.

Closes #27313 from tgravescs/SPARK-29148.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-02-12 16:45:42 -06:00
Shixiong Zhu e2ebca733c [SPARK-30779][SS] Fix some API issues found when reviewing Structured Streaming API docs
### What changes were proposed in this pull request?

- Fix the scope of `Logging.initializeForcefully` so that it doesn't appear in subclasses' public methods. Right now, `sc.initializeForcefully(false, false)` is allowed to called.
- Don't show classes under `org.apache.spark.internal` package in API docs.
- Add missing `since` annotation.
- Fix the scope of `ArrowUtils` to remove it from the API docs.

### Why are the changes needed?

Avoid leaking APIs unintentionally in Spark 3.0.0.

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

No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0.

### How was this patch tested?

Manually generated the API docs and verified the above issues have been fixed.

Closes #27528 from zsxwing/audit-ss-apis.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2020-02-10 14:26:14 -08:00
zhengruifeng 12e1bbaddb Revert "[SPARK-30642][SPARK-30659][SPARK-30660][SPARK-30662]"
### What changes were proposed in this pull request?
Revert
#27360
#27396
#27374
#27389

### Why are the changes needed?
BLAS need more performace tests, specially on sparse datasets.
Perfermance test of LogisticRegression (https://github.com/apache/spark/pull/27374) on sparse dataset shows that blockify vectors to matrices and use BLAS will cause performance regression.
LinearSVC and LinearRegression were also updated in the same way as LogisticRegression, so we need to revert them to make sure no regression.

### Does this PR introduce any user-facing change?
remove newly added param blockSize

### How was this patch tested?
reverted testsuites

Closes #27487 from zhengruifeng/revert_blockify_ii.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2020-02-08 08:46:16 +08:00
Yuanjian Li d8613571bc [SPARK-26700][CORE][FOLLOWUP] Add config spark.network.maxRemoteBlockSizeFetchToMem
### What changes were proposed in this pull request?
Add new config `spark.network.maxRemoteBlockSizeFetchToMem` fallback to the old config `spark.maxRemoteBlockSizeFetchToMem`.

### Why are the changes needed?
For naming consistency.

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

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

Closes #27463 from xuanyuanking/SPARK-26700-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 20:53:44 +08:00
yi.wu aebabf0bed [SPARK-30729][CORE] Eagerly filter out zombie TaskSetManager before offering resources
### What changes were proposed in this pull request?

Eagerly filter out zombie `TaskSetManager` before offering resources to reduce any overhead as possible.

And this PR also avoid doing `recomputeLocality` and `addPendingTask` when `TaskSetManager` is zombie.

### Why are the changes needed?

Zombie `TaskSetManager` could still exist in Pool's `schedulableQueue` when it has running tasks. Offering resources on a zombie `TaskSetManager` could bring unnecessary overhead and is meaningless.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27455 from Ngone51/exclude-zombie-tsm.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-06 12:48:27 +08:00
yi.wu 30e418a6fe [SPARK-30594][CORE] Do not post SparkListenerBlockUpdated when updateBlockInfo returns false
### What changes were proposed in this pull request?

If `updateBlockInfo` returns false, which means the `BlockManager` will re-register and report all blocks later. So, we may report two times for the same block, which causes `AppStatusListener` to count used memory for two times, too. As a result, the used memory can exceed the total memory.
So, this PR changes it to not post `SparkListenerBlockUpdated` when `updateBlockInfo` returns false. And, always clean up used memory whenever `AppStatusListener` receives `SparkListenerBlockManagerAdded`.

### Why are the changes needed?

This PR tries to fix negative memory usage in UI (https://user-images.githubusercontent.com/3488126/72131225-95e37e00-33b6-11ea-8708-6e5ed328d1ca.png, see #27144 ). Though, I'm not very sure this is the root cause for #27144 since known information is limited here.

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

No.

### How was this patch tested?

Added new tests by xuanyuanking

Closes #27306 from Ngone51/fix-possible-negative-memory.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-05 16:15:44 +08:00
Thomas Graves 878094f972 [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling
### What changes were proposed in this pull request?

This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment.
This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource.

I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs.

I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment.

The user would configured this like:
--conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin"

Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission.

Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources.

### Why are the changes needed?

to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments.

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

Yes a plugin api

### How was this patch tested?

Unit tests added and manual testing done on yarn and standalone modes.

Closes #27410 from tgravescs/hadoop27spark3.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 22:20:28 -06:00
Maxim Gekk 2d4b5eaee4 [SPARK-30676][CORE][TESTS] Eliminate warnings from deprecated constructors of java.lang.Integer and java.lang.Double
### What changes were proposed in this pull request?
- Replace `new Integer(0)` by a serializable instance in RDD.scala
- Use `.valueOf()` instead of constructors of `java.lang.Integer` and `java.lang.Double` because constructors has been deprecated, see https://docs.oracle.com/javase/9/docs/api/java/lang/Integer.html

### Why are the changes needed?
This fixes the following warnings:
1. RDD.scala:240: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
2. MutableProjectionSuite.scala:63: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
3. UDFSuite.scala:446: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information.
4. UDFSuite.scala:451: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.
5. HiveUserDefinedTypeSuite.scala:71: constructor Double in class Double is deprecated: see corresponding Javadoc for more information.

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

### How was this patch tested?
- By RDDSuite, MutableProjectionSuite, UDFSuite and HiveUserDefinedTypeSuite

Closes #27399 from MaxGekk/eliminate-warning-part4.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-31 15:03:16 -06:00
Wing Yew Poon 387ce89a06 [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration
### What changes were proposed in this pull request?

Add a section to the Configuration page to document configurations for executor metrics.
At the same time, rename spark.eventLog.logStageExecutorProcessTreeMetrics.enabled to spark.executor.processTreeMetrics.enabled and make it independent of spark.eventLog.logStageExecutorMetrics.enabled.

### Why are the changes needed?

Executor metrics are new in Spark 3.0. They lack documentation.
Memory metrics as a whole are always collected, but the ones obtained from the process tree have to be optionally enabled. Making this depend on a single configuration makes for more intuitive behavior. Given this, the configuration property is renamed to better reflect its meaning.

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

Yes, only in that the configurations are all new to 3.0.

### How was this patch tested?

Not necessary.

Closes #27329 from wypoon/SPARK-27324.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2020-01-31 14:28:02 -06:00
Jungtaek Lim (HeartSaVioR) 5e0faf9a3d [SPARK-29779][SPARK-30479][CORE][SQL][FOLLOWUP] Reflect review comments on post-hoc review
### What changes were proposed in this pull request?

This PR reflects review comments on post-hoc review among PRs for SPARK-29779 (#27085), SPARK-30479 (#27164). The list of review comments this PR addresses are below:

* https://github.com/apache/spark/pull/27085#discussion_r373304218
* https://github.com/apache/spark/pull/27164#discussion_r373300793
* https://github.com/apache/spark/pull/27164#discussion_r373301193
* https://github.com/apache/spark/pull/27164#discussion_r373301351

I also applied review comments to the CORE module (BasicEventFilterBuilder.scala) as well, as the review comments for SQL/core module (SQLEventFilterBuilder.scala) can be applied there as well.

### Why are the changes needed?

There're post-hoc reviews on PRs for such issues, like links in above section.

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

No

### How was this patch tested?

Existing UTs.

Closes #27414 from HeartSaVioR/SPARK-28869-SPARK-29779-SPARK-30479-FOLLOWUP-posthoc-reviews.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 10:17:07 -08:00
Thomas Graves ff0f636279 [SPARK-30638][CORE][FOLLOWUP] Fix a spacing issue and use UTF-8 instead of ASCII
### What changes were proposed in this pull request?

Followup from https://github.com/apache/spark/pull/27367 to fix a couple of my minor issues with the Test. Fix an indentation and then use UTF-8 instead of ASCII.

### Why are the changes needed?

followup

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

no

### How was this patch tested?

compiled and ran unit test

Closes #27420 from tgravescs/SPARK-30638-followup.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-31 09:48:34 -08:00
zebingl@fb.com 21bc0474bb [SPARK-30511][SPARK-28403][CORE] Don't treat failed/killed speculative tasks as pending in ExecutorAllocationManager
### What changes were proposed in this pull request?

Currently, when speculative tasks fail/get killed, they are still considered as pending and count towards the calculation of number of needed executors. To be more accurate: `stageAttemptToNumSpeculativeTasks(stageAttempt)` is incremented on onSpeculativeTaskSubmitted, but never decremented.  `stageAttemptToNumSpeculativeTasks -= stageAttempt` is performed on stage completion. **This means Spark is marking ended speculative tasks as pending, which leads to Spark to hold more executors that it actually needs!**

This PR fixes this issue by updating `stageAttemptToSpeculativeTaskIndices` and  `stageAttemptToNumSpeculativeTasks` on speculative tasks completion.  This PR also addresses some other minor issues: scheduler behavior after receiving an intentionally killed task event; try to address [SPARK-28403](https://issues.apache.org/jira/browse/SPARK-28403).

### Why are the changes needed?

This has caused resource wastage in our production with speculation enabled. With aggressive speculation, we found data skewed jobs can hold hundreds of idle executors with less than 10 tasks running.

An easy repro of the issue (`--conf spark.speculation=true --conf spark.executor.cores=4 --conf spark.dynamicAllocation.maxExecutors=1000` in cluster mode):
```
val n = 4000
val someRDD = sc.parallelize(1 to n, n)
someRDD.mapPartitionsWithIndex( (index: Int, it: Iterator[Int]) => {
if (index < 300 && index >= 150) {
    Thread.sleep(index * 1000) // Fake running tasks
} else if (index == 300) {
    Thread.sleep(1000 * 1000) // Fake long running tasks
}
it.toList.map(x => index + ", " + x).iterator
}).collect
```
You will see when running the last task, we would be hold 38 executors (see below), which is exactly (152 + 3) / 4 = 38.
![image](https://user-images.githubusercontent.com/9404831/72469112-9a7fac00-3793-11ea-8f50-74d0ab7325a4.png)

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

No

### How was this patch tested?

Added a comprehensive unit test.

Test with the above repro shows that we are holding 2 executors at the end
![image](https://user-images.githubusercontent.com/9404831/72469177-bbe09800-3793-11ea-850f-4a2c67142899.png)

Closes #27223 from linzebing/speculation_fix.

Authored-by: zebingl@fb.com <zebingl@fb.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 08:49:34 -06:00
Thomas Graves 3d2b8d8b13 [SPARK-30638][CORE] Add resources allocated to PluginContext
### What changes were proposed in this pull request?

Add the allocated resources to parameters to the PluginContext so that any plugins in driver or executor could use this information to initialize devices or use this information in a useful manner.

### Why are the changes needed?

To allow users to initialize/track devices once at the executor level before each task runs to use them.

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

Yes to the people using the Executor/Driver plugin interface.

### How was this patch tested?

Unit tests and manually by writing a plugin that initialized GPU's using this interface.

Closes #27367 from tgravescs/pluginWithResources.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 08:25:32 -06:00
Jungtaek Lim (HeartSaVioR) ca3a64bffb [SPARK-30481][CORE][FOLLOWUP] Execute log compaction only when merge application listing is successful
### What changes were proposed in this pull request?

This PR fixes a couple of minor issues on SPARK-30481:

* SHS runs "compaction" regardless of the result of "merge application listing".

If "merge application listing" fails, most likely the application log will have some issue and "compaction" won't work properly then. We can just skip trying compaction when "merge application listing" fails.

* When "compaction" throws exception we don't handle it.

It's expected to swallow exception, but we don't even log the exception for now. It should be logged properly.

### Why are the changes needed?

Described in above section.

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

No.

### How was this patch tested?

Existing UTs.

Closes #27408 from HeartSaVioR/SPARK-30481-FOLLOWUP-MINOR-FIXES.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-30 21:04:08 -08:00
Thomas Graves e5c7f89082 [SPARK-30529][CORE] Improve error messages when Executor dies before registering with driver
…

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

If the resource discovery goes bad, like it doesn't return enough GPUs,  currently it just throws an exception. This is hard for users to see because you have to go find the executor logs and its not reported back to the driver.  On yarn if you exit explicitly then the driver logs show the error thrown and its much more useful.  On yarn with the explicit exit with non-zero it also goes against failed executor launch attempts and the application will eventually exit. so if its fundamentally a bad configuration or bad discovery script it won't just hang forever.   I also tested on k8s and standalone and the behaviors there don't change, the executor cleanly exit with an error message in the logs. The standalone ui makes it easy to see failed executors.

### Why are the changes needed?

better user experience.

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

no api changes

### How was this patch tested?

ran unit tests and manually tested on yarn, standalone, and k8s.

Closes #27385 from tgravescs/SPARK-30529.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-29 15:37:11 -08:00
uncleGen 7173786153
[SPARK-29543][SS][UI] Structured Streaming Web UI
### What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:
   - "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries.
  - "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including `Input Rate`, `Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration`

![Screen Shot 2020-01-29 at 1 38 00 PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png)
![Screen Shot 2020-01-29 at 1 39 16 PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png)

### Why are the changes needed?

It helps users to better monitor Structured Streaming query.

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

No

### How was this patch tested?

- new added and existing UTs
- manual test

Closes #26201 from uncleGen/SPARK-29543.

Lead-authored-by: uncleGen <hustyugm@gmail.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Genmao Yu <hustyugm@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-29 13:43:51 -08:00
Saurabh Chawla d0f635e3bc [SPARK-30582][WEBUI] Spark UI is not showing Aggregated Metrics by Executor in stage page
### What changes were proposed in this pull request?

There are scenarios where Spark History Server is located behind the VPC. So whenever api calls hit to get the executor Summary(allexecutors). There can be delay in getting the response of executor summary and in mean time "stage-page-template.html" is loaded and the response of executor Summary is not added to the stage-page-template.html.

As the result of which Aggregated Metrics by Executor in stage page is showing blank.

This scenario can be easily found in the cases when there is some proxy-server which is responsible for sending the request and response to spark History server.
This can be reproduced in Knox/In-house proxy servers which are used to send and receive response to Spark History Server.

Alternative scenario to test this case, Open the spark UI in developer mode in browser add some breakpoint in stagepage.js, this will add some delay in getting the response and now if we check the spark UI for stage Aggregated Metrics by Executor in stage page is showing blank.

So In-order to fix this there is a need to add the change in stagepage.js . There is a need to add the api call to get the html page(stage-page-template.html) first and after that other api calls to get the data that needs to attached in the stagepage (like executor Summary, stageExecutorSummaryInfoKeys exc)

### Why are the changes needed?
Since stage page is useful for debugging purpose, This helps in understanding how many task ran on the particular executor and information related to shuffle read and write on that executor.

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

### How was this patch tested?
Manually tested. Testing this in a reproducible way requires a running browser or HTML rendering engine that executes the JavaScript.Open the spark UI in developer mode in browser add some breakpoint in stagepage.js, this will add some delay in getting the response and now if we check the spark UI for stage Aggregated Metrics by Executor in stage page is showing blank.

Before fix

<img width="1529" alt="Screenshot 2020-01-20 at 3 21 55 PM" src="https://user-images.githubusercontent.com/34540906/72716739-bcfd3500-3b98-11ea-8dbe-90a135822f92.png">

After fix

<img width="1540" alt="Screenshot 2020-01-20 at 3 23 12 PM" src="https://user-images.githubusercontent.com/34540906/72716782-d30af580-3b98-11ea-8764-2bde77764604.png">

Closes #27292 from SaurabhChawla100/SPARK-30582.

Authored-by: Saurabh Chawla <saurabhc@qubole.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-29 08:49:45 -06:00
Jungtaek Lim (HeartSaVioR) a2fe73b83c [SPARK-30481][CORE] Integrate event log compactor into Spark History Server
### What changes were proposed in this pull request?

This patch addresses remaining functionality on event log compaction: integrate compaction into FsHistoryProvider.

This patch is next task of SPARK-30479 (#27164), please refer the description of PR #27085 to see overall rationalization of this patch.

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

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

No.

### How was this patch tested?

Added UT.

Closes #27208 from HeartSaVioR/SPARK-30481.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@apache.org>
2020-01-28 17:16:21 -08:00
zhengruifeng 96d27274f5 [SPARK-30642][ML][PYSPARK] LinearSVC blockify input vectors
### What changes were proposed in this pull request?
1, stack input vectors to blocks (like ALS/MLP);
2, add new param `blockSize`;
3, add a new class `InstanceBlock`
4, standardize the input outside of optimization procedure;

### Why are the changes needed?
1, reduce RAM to persist traing dataset; (save ~40% in test)
2, use Level-2 BLAS routines; (12% ~ 28% faster, without native BLAS)

### Does this PR introduce any user-facing change?
a new param `blockSize`

### How was this patch tested?
existing and updated testsuites

Closes #27360 from zhengruifeng/blockify_svc.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2020-01-28 20:55:21 +08:00
Udbhav30 84f11548e4 [SPARK-30604][CORE] Fix a log message by including hostLocalBlockBytes to total bytes
### What changes were proposed in this pull request?

 Add HostLocalBlock size in log total bytes

### Why are the changes needed?
total size in log is wrong as hostlocal block size is missed

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

### How was this patch tested?
Manually checking the log

Closes #27320 from Udbhav30/bug.

Authored-by: Udbhav30 <u.agrawal30@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-22 14:20:28 -08:00
Maxim Gekk a131031f95 [SPARK-30599][CORE][TESTS] Increase the maximum number of log events in LogAppender
### What changes were proposed in this pull request?
Increased the limit for log events that could be stored in `SparkFunSuite.LogAppender` from 100 to 1000.

### Why are the changes needed?
Sometimes (see traces in SPARK-30599) additional info is logged via log4j, and appended to `LogAppender`. For example, unusual log entries are:
```
[36] Removed broadcast_214_piece0 on 192.168.1.66:52354 in memory (size: 5.7 KiB, free: 2003.8 MiB)
[37] Removed broadcast_204_piece0 on 192.168.1.66:52354 in memory (size: 5.7 KiB, free: 2003.9 MiB)
[38] Removed broadcast_200_piece0 on 192.168.1.66:52354 in memory (size: 3.7 KiB, free: 2003.9 MiB)
[39] Removed broadcast_207_piece0 on 192.168.1.66:52354 in memory (size: 24.2 KiB, free: 2003.9 MiB)
[40] Removed broadcast_208_piece0 on 192.168.1.66:52354 in memory (size: 24.2 KiB, free: 2003.9 MiB)
```
and a test which uses `LogAppender` can fail with the exception:
```
java.lang.IllegalStateException: Number of events reached the limit of 100 while logging CSV header matches to schema w/ enforceSchema.
```

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

### How was this patch tested?
By re-running `"SPARK-23786: warning should be printed if CSV header doesn't conform to schema"` in a loop.

Closes #27312 from MaxGekk/log-appender-filter.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-21 14:27:55 -08:00
Wenchen Fan 595cdb09a4 [SPARK-30571][CORE] fix splitting shuffle fetch requests
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26930 to fix a bug.

When we create shuffle fetch requests, we first collect blocks until they reach the max size. Then we try to merge the blocks (the batch shuffle fetch feature) and split the merged blocks to several groups, to make sure each group doesn't reach the max numBlocks. For the last group, if it's smaller than the max numBlocks, put it back to the input list and deal with it again later.

The last step has a problem:
1. if we put a merged block back to the input list and merge it again, it fails.
2. when putting back some blocks, we should update `numBlocksToFetch`

This PR fixes these 2 problems.

### Why are the changes needed?
bug fix

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

no

### How was this patch tested?

new test

Closes #27280 from cloud-fan/aqe.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 14:45:50 +08:00
Maxim Gekk fd69533593 [SPARK-30482][CORE][SQL][TESTS][FOLLOW-UP] Output caller info in log appenders while reaching the limit
### What changes were proposed in this pull request?
In the PR, I propose to output additional msg from the tests where a log appender is added. The message is printed as a part of `IllegalStateException` in the case of reaching the limit of maximum number of logged events.

### Why are the changes needed?
If a log appender is not removed from the log4j appenders list. the caller message could help to investigate the problem and find the test which doesn't remove the log appender.

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

### How was this patch tested?
By running the modified test suites `AvroSuite`, `CSVSuite`, `ResolveHintsSuite` and etc.

Closes #27296 from MaxGekk/assign-name-to-log-appender.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-21 10:19:07 +09:00
Kousuke Saruta 3858e94ef9 [SPARK-30566][BUILD] Iterator doesn't refer outer identifier named "iterator" properly in Scala 2.13
### What changes were proposed in this pull request?

Renamed an identifier `iterator` to `iter` to avoid compile error with Scala 2.13.

### Why are the changes needed?

As of Scala 2.13, scala.collection.Iterator has "iterator" method so if an inner class of Iterator means to refer an outer identifier named "iterator", it does not work as we think.
I listed source files that can be affected by that change by `find . -name "*.scala" -exec grep -El "new .*Iterator\[.* +{"  {} \;`
As far as I confirmed util.Utils` is affected.

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

No.

### How was this patch tested?

Existing tests.

Closes #27275 from sarutak/fix-iterator-for-2.13.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-20 10:11:41 +09:00
git f5f05d549e [SPARK-30310][CORE] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
### What changes were proposed in this pull request?
1) Added missing match case to SparkUncaughtExceptionHandler, so that it would not halt the process when the exception doesn't match any of the match case statements.
2) Added log message before halting process.  During debugging it wasn't obvious why the Worker process would DEAD (until we set SPARK_NO_DAEMONIZE=1) due to the shell-scripts puts the process into background and essentially absorbs the exit code.
3) Added SparkUncaughtExceptionHandlerSuite.  Basically we create a Spark exception-throwing application with SparkUncaughtExceptionHandler and then check its exit code.

### Why are the changes needed?
SPARK-30310, because the process would halt unexpectedly.

### How was this patch tested?
All unit tests (mvn test) were ran and OK.

Closes #26955 from tinhto-000/uncaught_exception_fix.

Authored-by: git <tinto@us.ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-17 09:46:29 -06:00
Thomas Graves 6dbfa2bb9c [SPARK-29306][CORE] Stage Level Sched: Executors need to track what ResourceProfile they are created with
### What changes were proposed in this pull request?

This is the second PR for the Stage Level Scheduling. This is adding in the necessary executor side changes:
1) executors to know what ResourceProfile they should be using
2) handle parsing the resource profile settings - these are not in the global configs
3) then reporting back to the driver what resource profile it was started with.

This PR adds all the piping for YARN to pass the information all the way to executors, but it just uses the default ResourceProfile (which is the global applicatino level configs).

At a high level these changes include:
1) adding a new --resourceProfileId option to the CoarseGrainedExecutorBackend
2) Add the ResourceProfile settings to new internal confs that gets passed into the Executor
3) Executor changes that use the resource profile id passed in to read the corresponding ResourceProfile confs and then parse those requests and discover resources as necessary
4) Executor registers to Driver with the Resource profile id so that the ExecutorMonitor can track how many executor with each profile are running
5) YARN side changes to show that passing the resource profile id and confs actually works. Just uses the DefaultResourceProfile for now.

I also removed a check from the CoarseGrainedExecutorBackend that used to check to make sure there were task requirements before parsing any custom resource executor requests.  With the resource profiles this becomes much more expensive because we would then have to pass the task requests to each executor and the check was just a short cut and not really needed. It was much cleaner just to remove it.

Note there were some changes to the ResourceProfile, ExecutorResourceRequests, and TaskResourceRequests in this PR as well because I discovered some issues with things not being immutable. That api now look like:

val rpBuilder = new ResourceProfileBuilder()
val ereq = new ExecutorResourceRequests()
val treq = new TaskResourceRequests()

ereq.cores(2).memory("6g").memoryOverhead("2g").pysparkMemory("2g").resource("gpu", 2, "/home/tgraves/getGpus")
treq.cpus(2).resource("gpu", 2)

val resourceProfile = rpBuilder.require(ereq).require(treq).build

This makes is so that ResourceProfile is immutable and Spark can use it directly without worrying about the user changing it.

### Why are the changes needed?

These changes are needed for the executor to report which ResourceProfile they are using so that ultimately the dynamic allocation manager can use that information to know how many with a profile are running and how many more it needs to request.  Its also needed to get the resource profile confs to the executor so that it can run the appropriate discovery script if needed.

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

No

### How was this patch tested?

Unit tests and manually on YARN.

Closes #26682 from tgravescs/SPARK-29306.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-17 08:15:25 -06:00
Marcelo Vanzin dca838058f [SPARK-29950][K8S] Blacklist deleted executors in K8S with dynamic allocation
The issue here is that when Spark is downscaling the application and deletes
a few pod requests that aren't needed anymore, it may actually race with the
K8S scheduler, who may be bringing up those executors. So they may have enough
time to connect back to the driver, register, to just be deleted soon after.
This wastes resources and causes misleading entries in the driver log.

The change (ab)uses the blacklisting mechanism to consider the deleted excess
pods as blacklisted, so that if they try to connect back, the driver will deny
it.

It also changes the executor registration slightly, since even with the above
change there were misleading logs. That was because the executor registration
message was an RPC that always succeeded (bar network issues), so the executor
would always try to send an unregistration message to the driver, which would
then log several messages about not knowing anything about the executor. The
change makes the registration RPC succeed or fail directly, instead of using
the separate failure message that would lead to this issue.

Note the last change required some changes in a standalone test suite related
to dynamic allocation, since it relied on the driver not throwing exceptions
when a duplicate executor registration happened.

Tested with existing unit tests, and with live cluster with dyn alloc on.

Closes #26586 from vanzin/SPARK-29950.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-16 13:37:11 -08:00
zhengruifeng aec55cd1ca [SPARK-30502][ML][CORE] PeriodicRDDCheckpointer support storageLevel
### What changes were proposed in this pull request?
1, add field `storageLevel` in `PeriodicRDDCheckpointer`
2, for ml.GBT/ml.RF set storageLevel=`StorageLevel.MEMORY_AND_DISK`

### Why are the changes needed?
Intermediate RDDs in ML are cached with storageLevel=StorageLevel.MEMORY_AND_DISK.
PeriodicRDDCheckpointer & PeriodicGraphCheckpointer now store RDD with storageLevel=StorageLevel.MEMORY_ONLY, it maybe nice to set the storageLevel of checkpointer.

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

### How was this patch tested?
existing testsuites

Closes #27189 from zhengruifeng/checkpointer_storage.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2020-01-16 11:01:30 +08:00
Gabor Somogyi 6c178a5d16 [SPARK-30495][SS] Consider spark.security.credentials.kafka.enabled and cluster configuration when checking latest delegation token
### What changes were proposed in this pull request?
Spark SQL Kafka consumer connector considers delegation token usage even if the user configures `sasl.jaas.config` manually.

In this PR I've added `spark.security.credentials.kafka.enabled` and cluster configuration check to the condition.

### Why are the changes needed?
Now it's not possible to configure `sasl.jaas.config` manually.

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

### How was this patch tested?
Existing + additional unit tests.

Closes #27191 from gaborgsomogyi/SPARK-30495.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 11:46:34 -08:00
Jungtaek Lim (HeartSaVioR) e751bc66a0 [SPARK-30479][SQL] Apply compaction of event log to SQL events
### What changes were proposed in this pull request?

This patch addresses adding event filter to handle SQL related events. This patch is next task of SPARK-29779 (#27085), please refer the description of PR #27085 to see overall rationalization of this patch.

Below functionalities will be addressed in later parts:

* integrate compaction into FsHistoryProvider
* documentation about new configuration

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

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

No.

### How was this patch tested?

Added UTs.

Closes #27164 from HeartSaVioR/SPARK-30479.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 10:47:31 -08:00
jiake a2aa966ef6 [SPARK-29544][SQL] optimize skewed partition based on data size
### What changes were proposed in this pull request?
Skew Join is common and can severely downgrade performance of queries, especially those with joins. This PR aim to optimization the skew join based on the runtime Map output statistics by adding "OptimizeSkewedPartitions" rule. And The details design doc is [here](https://docs.google.com/document/d/1NkXN-ck8jUOS0COz3f8LUW5xzF8j9HFjoZXWGGX2HAg/edit). Currently we can support "Inner, Cross, LeftSemi, LeftAnti, LeftOuter, RightOuter" join type.

### Why are the changes needed?
To optimize the skewed partition in runtime based on AQE

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

### How was this patch tested?
UT

Closes #26434 from JkSelf/skewedPartitionBasedSize.

Lead-authored-by: jiake <ke.a.jia@intel.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: JiaKe <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 20:31:44 +08:00
yu 4462756216 [SPARK-30325][CORE] markPartitionCompleted cause task status inconsistent
### **What changes were proposed in this pull request?**
 Fix task status inconsistent in `executorLost` which caused by `markPartitionCompleted`

### **Why are the changes needed?**
The inconsistent will cause app hung up.
The bugs occurs in the corer case as follows:
1. The stage occurs during stage retry, scheduler will resubmit a new stage with unfinished tasks.
2. Those unfinished tasks in origin stage finished and the same task on the new retry stage hasn't finished, it will mark the task partition on the current retry stage as succesuful in TSM `successful` array variable.
3. The executor crashed when it is running tasks which have succeeded by origin stage, it cause TSM run `executorLost` to rescheduler the task on the executor, and it will change the partition's running status in `copiesRunning` twice to -1.
4. 'dequeueTaskFromList' will use `copiesRunning` equal 0 as reschedule basis when rescheduler tasks, and now it is -1, can't to reschedule, and the app will hung forever.

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

### **How was this patch tested?**

Closes #26975 from seayoun/fix_stageRetry_executorCrash_cause_problems.

Authored-by: yu <you@example.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 17:17:13 +08:00
Maxim Gekk 88fc8dbc09 [SPARK-30482][SQL][CORE][TESTS] Add sub-class of AppenderSkeleton reusable in tests
### What changes were proposed in this pull request?
In the PR, I propose to define a sub-class of `AppenderSkeleton` in `SparkFunSuite` and reuse it from other tests. The class stores incoming `LoggingEvent` in an array which is available to tests for future analysis of logged events.

### Why are the changes needed?
This eliminates code duplication in tests.

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

### How was this patch tested?
By existing test suites - `CSVSuite`, `OptimizerLoggingSuite`, `JoinHintSuite`, `CodeGenerationSuite` and `SQLConfSuite`.

Closes #27166 from MaxGekk/dedup-appender-skeleton.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2020-01-14 16:03:10 +09:00
Neal Song 65b603d597 [SPARK-30458][WEBUI] Fix Wrong Executor Computing Time in Time Line of Stage Page
### What changes were proposed in this pull request?
The Executor Computing Time in Time Line of Stage Page will be right

### Why are the changes needed?
The Executor Computing Time in Time Line of Stage Page is Wrong. It includes the Scheduler Delay Time, while the Proportion excludes the Scheduler Delay

<img width="1467" alt="Snipaste_2020-01-08_19-04-33" src="https://user-images.githubusercontent.com/3488126/71976714-f2795880-3251-11ea-869a-43ca6e0cf96a.png">

The right executor computing time is 1ms, but the number in UI is 3ms(include 2ms scheduler delay); the proportion is right.

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

### How was this patch tested?
Manual

Closes #27135 from sddyljsx/SPARK-30458.

Lead-authored-by: Neal Song <neal_song@126.com>
Co-authored-by: neal_song <neal_song@126.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-11 20:08:46 -08:00
Neal Song 26ad8f8f34 [SPARK-30478][CORE][DOCS] Fix Memory Package documentation
### What changes were proposed in this pull request?
update the doc of momery package

### Why are the changes needed?
From Spark 2.0, the storage memory also uses off heap memory. We update the doc here.
![memory manager](https://user-images.githubusercontent.com/3488126/72124682-9b35ce00-33a0-11ea-8cf9-301494974ef4.png)

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

### How was this patch tested?
No Tests Needed

Closes #27160 from sddyljsx/SPARK-30478.

Lead-authored-by: Neal Song <neal_song@126.com>
Co-authored-by: neal_song <neal_song@126.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-11 19:51:52 -08:00
Jungtaek Lim (HeartSaVioR) 7fb17f5943 [SPARK-29779][CORE] Compact old event log files and cleanup
### What changes were proposed in this pull request?

This patch proposes to compact old event log files when end users enable rolling event log, and clean up these files after compaction.

Here the "compaction" really mean is filtering out listener events for finished/removed things - like jobs which take most of space for event log file except SQL related events. To achieve this, compactor does two phases reading: 1) tracking the live jobs (and more to add) 2) filtering events via leveraging the information about live things and rewriting to the "compacted" file.

This approach retains the ability of compatibility on event log file and adds the possibility of reducing the overall size of event logs. There's a downside here as well: executor metrics for tasks would be inaccurate, as compactor will filter out the task events which job is finished, but I don't feel it as a blocker.

Please note that SPARK-29779 leaves below functionalities for future JIRA issue as the patch for SPARK-29779 is too huge and we decided to break down:

* apply filter in SQL events
* integrate compaction into FsHistoryProvider
* documentation about new configuration

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

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

No.

### How was this patch tested?

Added UTs.

Closes #27085 from HeartSaVioR/SPARK-29779-part1.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-10 09:52:59 -08:00
Thomas Graves d6532c7079 [SPARK-30448][CORE] accelerator aware scheduling enforce cores as limiting resource
### What changes were proposed in this pull request?

This PR is to make sure cores is the limiting resource when using accelerator aware scheduling and fix a few issues with SparkContext.checkResourcesPerTask

For the first version of accelerator aware scheduling(SPARK-27495), the SPIP had a condition that we can support dynamic allocation because we were going to have a strict requirement that we don't waste any resources. This means that the number of slots each executor has could be calculated from the number of cores and task cpus just as is done today.

Somewhere along the line of development we relaxed that and only warn when we are wasting resources. This breaks the dynamic allocation logic if the limiting resource is no longer the cores because its using the cores and task cpus to calculate the number of executors it needs.  This means we will request less executors then we really need to run everything. We have to enforce that cores is always the limiting resource so we should throw if its not.

The only issue with us enforcing this is on cluster managers (standalone and mesos coarse grained) where we don't know the executor cores up front by default. Meaning the spark.executor.cores config defaults to 1 but when the executor is started by default it gets all the cores of the Worker. So we have to add logic specifically to handle that and we can't enforce this requirements, we can just warn when dynamic allocation is enabled for those.

### Why are the changes needed?

Bug in dynamic allocation if cores is not limiting resource and warnings not correct.

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

no

### How was this patch tested?

Unit test added and manually tested the confiditions on local mode, local cluster mode, standalone mode, and yarn.

Closes #27138 from tgravescs/SPARK-30446.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-10 08:32:28 -06:00
Ajith 18daa37cdb [SPARK-30440][CORE][TESTS] Avoid race condition in TaskSetManagerSuite by not using resourceOffer
### What changes were proposed in this pull request?
There is a race condition in test case introduced in SPARK-30359 between reviveOffers in org.apache.spark.scheduler.TaskSchedulerImpl#submitTasks and org.apache.spark.scheduler.TaskSetManager#resourceOffer, in the testcase

No need to do resourceOffers as submitTask will revive offers from task set

### Why are the changes needed?
Fix flaky test

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

### How was this patch tested?
Test case can pass after the change

Closes #27115 from ajithme/testflaky.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-08 16:28:19 -08:00
Yuchen Huo c49abf820d [SPARK-30417][CORE] Task speculation numTaskThreshold should be greater than 0 even EXECUTOR_CORES is not set under Standalone mode
### What changes were proposed in this pull request?

Previously in https://github.com/apache/spark/pull/26614/files#diff-bad3987c83bd22d46416d3dd9d208e76R90, we compare the number of tasks with `(conf.get(EXECUTOR_CORES) / sched.CPUS_PER_TASK)`. In standalone mode if the value is not explicitly set by default, the conf value would be 1 but the executor would actually use all the cores of the worker. So it is allowed to have `CPUS_PER_TASK` greater than `EXECUTOR_CORES`. To handle this case, we change the condition to be `numTasks <= Math.max(conf.get(EXECUTOR_CORES) / sched.CPUS_PER_TASK, 1)`

### Why are the changes needed?

For standalone mode if the user set the `spark.task.cpus` to be greater than 1 but didn't set the `spark.executor.cores`. Even though there is only 1 task in the stage it would not be speculative run.

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

Solve the problem above by allowing speculative run when there is only 1 task in the stage.

### How was this patch tested?

Existing tests and one more test in TaskSetManagerSuite

Closes #27126 from yuchenhuo/SPARK-30417.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2020-01-08 11:30:32 -08:00
Thomas Graves 0a72dba6f5 [SPARK-30445][CORE] Accelerator aware scheduling handle setting configs to 0
### What changes were proposed in this pull request?

Handle the accelerator aware configs being set to 0. This PR will just ignore the requests when the amount is 0.

### Why are the changes needed?

Better user experience

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

no

### How was this patch tested?

Unit tests added and manually tested on yarn, standalone, local, k8s.

Closes #27118 from tgravescs/SPARK-30445.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-08 09:13:48 -08:00
zhengruifeng a93b996635 [MINOR][ML][INT] Array.fill(0) -> Array.ofDim; Array.empty -> Array.emptyIntArray
### What changes were proposed in this pull request?
1, for primitive types `Array.fill(n)(0)` -> `Array.ofDim(n)`;
2, for `AnyRef` types `Array.fill(n)(null)` -> `Array.ofDim(n)`;
3, for primitive types `Array.empty[XXX]` -> `Array.emptyXXXArray`

### Why are the changes needed?
`Array.ofDim` avoid assignments;
`Array.emptyXXXArray` avoid create new object;

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

### How was this patch tested?
existing testsuites

Closes #27133 from zhengruifeng/minor_fill_ofDim.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-09 00:07:42 +09:00
yi.wu b3c2d735d4 [MINOR][CORE] Process bar should print new line to avoid polluting logs
### What changes were proposed in this pull request?

Use `println()` instead of `print()` to show process bar in console.

### Why are the changes needed?

Logs are polluted by process bar:

![image](https://user-images.githubusercontent.com/16397174/71623360-f59f9380-2c16-11ea-8e27-858a10caf1f5.png)

This is easy to reproduce:

1. start `./bin/spark-shell`
2. `sc.setLogLevel("INFO")`
3. run: `spark.range(100000000).coalesce(1).write.parquet("/tmp/result")`

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

Yeah, more friendly format in console.

### How was this patch tested?

Tested manually.

Closes #27061 from Ngone51/fix-processbar.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-08 09:06:20 -06:00
Jungtaek Lim (HeartSaVioR) 895e572b73 [SPARK-30313][CORE] Ensure EndpointRef is available MasterWebUI/WorkerPage
### What changes were proposed in this pull request?

This patch fixes flaky tests "master/worker web ui available" & "master/worker web ui available with reverseProxy" in MasterSuite.

Tracking back from stack trace below,

```
19/12/19 13:48:39.160 dispatcher-event-loop-4 INFO Worker: WorkerWebUI is available at http://localhost:8080/proxy/worker-20191219
134839-localhost-36054
19/12/19 13:48:39.296 WorkerUI-52072 WARN JettyUtils: GET /json/ failed: java.lang.NullPointerException
java.lang.NullPointerException
        at org.apache.spark.deploy.worker.ui.WorkerPage.renderJson(WorkerPage.scala:39)
        at org.apache.spark.ui.WebUI.$anonfun$attachPage$2(WebUI.scala:91)
        at org.apache.spark.ui.JettyUtils$$anon$1.doGet(JettyUtils.scala:80)
        at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
        at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
        at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:873)
        at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1623)
        at org.apache.spark.ui.HttpSecurityFilter.doFilter(HttpSecurityFilter.scala:95)
        at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1610)
        at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:540)
```

there's possible race condition in `Dispatcher.registerRpcEndpoint()`:

481fb63f97/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala (L64-L77)

`getMessageLoop()` initializes a new Inbox for this endpoint for both DedicatedMessageLoop
 and SharedMessageLoop, which calls `onStart()`  "asynchronously" and "eventually" via posting `OnStart` message. `onStart()` will initialize UI page instance(s), so the execution of `endpointRefs.put()` and initializing UI page instance(s) are "concurrent".

MasterPage and WorkerPage retrieve endpoint ref and store it as "val" assuming endpoint ref is valid when they're initialized - so in bad case they could store "null" as endpoint ref, and don't change.

481fb63f97/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala (L33-L38)

481fb63f97/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala (L35-L41)

This patch breaks down the step to `find the right message loop` and `register endpoint to message loop`, and ensure endpoint ref is set "before" registering endpoint to message loop.

### Why are the changes needed?

We observed the test failures from Jenkins; below are the links:

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115583/testReport/
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115700/testReport/

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

No.

### How was this patch tested?

Existing UTs.

You can also reproduce the bug consistently via adding `Thread.sleep(1000)` just before `endpointRefs.put(endpoint, endpointRef)` in `Dispatcher.registerRpcEndpoint(...)`.

Closes #27010 from HeartSaVioR/SPARK-30313.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-06 08:41:55 -08:00
yi.wu 4a093176ea [SPARK-30359][CORE] Don't clear executorsPendingToRemove at the beginning of CoarseGrainedSchedulerBackend.reset
### What changes were proposed in this pull request?

Remove `executorsPendingToRemove.clear()` from `CoarseGrainedSchedulerBackend.reset()`.

### Why are the changes needed?

Clear `executorsPendingToRemove` before remove executors will cause all tasks running on those "pending to remove" executors to count failures. But that's not true for the case of `executorsPendingToRemove(execId)=true`.

Besides, `executorsPendingToRemove` will be cleaned up within `removeExecutor()` at the end just as same as `executorsPendingLossReason`.

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

No

### How was this patch tested?

Added a new test in `TaskSetManagerSuite`.

Closes #27017 from Ngone51/dont-clear-eptr-in-reset.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-03 22:54:05 +08:00
Marcelo Vanzin cb9fc4bb6f [SPARK-30225][CORE] Correct read() behavior past EOF in NioBufferedFileInputStream
This bug manifested itself when another stream would potentially make a call
to NioBufferedFileInputStream.read() after it had reached EOF in the wrapped
stream. In that case, the refill() code would clear the output buffer the
first time EOF was found, leaving it in a readable state for subsequent
read() calls. If any of those calls were made, bad data would be returned.

By flipping the buffer before returning, even in the EOF case, you get the
correct behavior in subsequent calls. I picked that approach to avoid keeping
more state in this class, although it means calling the underlying stream
even after EOF (which is fine, but perhaps a little more expensive).

This showed up (at least) when using encryption, because the commons-crypto
StreamInput class does not track EOF internally, leaving it for the wrapped
stream to behave correctly.

Tested with added unit test + slightly modified test case attached to SPARK-18105.

Closes #27084 from vanzin/SPARK-30225.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-03 18:38:18 +09:00
07ARB 4f9d3dc6ba [SPARK-30384][WEBUI] Needs to improve the Column name and Add tooltips for the Fair Scheduler Pool Table
### What changes were proposed in this pull request?
Needs to improve the Column name and tooltips for the Fair Scheduler Pool Table.

### Why are the changes needed?
Need to correct SchedulingMode  column name to  -> 'Scheduling Mode' and tooltips need to add for Minimum Share, Pool Weight and Scheduling Mode (require meaning full Tool tips for the end user to understand.)

### Does this PR introduce any user-facing change?
YES
![Screenshot 2020-01-03 at 10 10 47 AM](https://user-images.githubusercontent.com/8948111/71707687-7aee9800-2e11-11ea-93cc-52df0b9114dd.png)

### How was this patch tested?
Manual Testing.

Closes #27047 from 07ARB/SPARK-30384.

Lead-authored-by: 07ARB <ankitrajboudh@gmail.com>
Co-authored-by: Ankitraj <8948111+07ARB@users.noreply.github.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-01-02 21:13:15 -08:00
Wang Shuo 10cae04108 [SPARK-30285][CORE] Fix deadlock between LiveListenerBus#stop and AsyncEventQueue#removeListenerOnError
### What changes were proposed in this pull request?

There is a deadlock between `LiveListenerBus#stop` and `AsyncEventQueue#removeListenerOnError`.

We can reproduce as follows:

1. Post some events to `LiveListenerBus`
2. Call `LiveListenerBus#stop` and hold the synchronized lock of `bus`(5e92301723/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala (L229)), waiting until all the events are processed by listeners, then remove all the queues
3. Event queue would drain out events by posting to its listeners. If a listener is interrupted, it will call `AsyncEventQueue#removeListenerOnError`,  inside it will call `bus.removeListener`(7b1b60c758/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala (L207)), trying to acquire synchronized lock of bus, resulting in deadlock

This PR  removes the `synchronized` from `LiveListenerBus.stop` because underlying data structures themselves are thread-safe.

### Why are the changes needed?
To fix deadlock.

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

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

Closes #26924 from wangshuo128/event-queue-race-condition.

Authored-by: Wang Shuo <wangshuo128@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-02 16:40:22 -08:00
Liang-Chi Hsieh 9eff1186ae [SPARK-30379][CORE] Avoid OOM when using collection accumulator
### What changes were proposed in this pull request?

This patch proposes to only convert first few elements of collection accumulators in `LiveEntityHelpers.newAccumulatorInfos`.

### Why are the changes needed?

One Spark job on our cluster uses collection accumulator to collect something and has encountered an exception like:

```
java.lang.OutOfMemoryError: Java heap space
    at java.util.Arrays.copyOf(Arrays.java:3332)
    at java.lang.AbstractStringBuilder.ensureCapacityInternal(AbstractStringBuilder.java:124)
    at java.lang.AbstractStringBuilder.append(AbstractStringBuilder.java:448)
    at java.lang.StringBuilder.append(StringBuilder.java:136)
    at java.lang.StringBuilder.append(StringBuilder.java:131)
    at java.util.AbstractCollection.toString(AbstractCollection.java:462)
    at java.util.Collections$UnmodifiableCollection.toString(Collections.java:1035)
    at org.apache.spark.status.LiveEntityHelpers$$anonfun$newAccumulatorInfos$2$$anonfun$apply$3.apply(LiveEntity.scala:596)
    at org.apache.spark.status.LiveEntityHelpers$$anonfun$newAccumulatorInfos$2$$anonfun$apply$3.apply(LiveEntity.scala:596)
    at scala.Option.map(Option.scala:146)
    at org.apache.spark.status.LiveEntityHelpers$$anonfun$newAccumulatorInfos$2.apply(LiveEntity.scala:596)
    at org.apache.spark.status.LiveEntityHelpers$$anonfun$newAccumulatorInfos$2.apply(LiveEntity.scala:591)
```

`LiveEntityHelpers.newAccumulatorInfos` converts `AccumulableInfo`s to `v1.AccumulableInfo` by calling `toString` on accumulator's value. For collection accumulator, it might take much more memory when in string representation, for example, collection accumulator of long values, and cause OOM (in this job, the driver memory is 6g).

Looks like the results of `newAccumulatorInfos` are used in api and ui. For such usage, it also does not make sense to have very long string of complete collection accumulators.

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

Yes. Collection accumulator now only shows first few elements in api and ui.

### How was this patch tested?

Unit test.

Manual test. Launched a Spark shell, ran:
```scala
val accum = sc.collectionAccumulator[Long]("Collection Accumulator Example")
sc.range(0, 10000, 1, 1).foreach(x => accum.add(x))
accum.value
```

<img width="2533" alt="Screen Shot 2019-12-30 at 2 03 43 PM" src="https://user-images.githubusercontent.com/68855/71602488-6eb2c400-2b0d-11ea-8725-dba36478198f.png">

Closes #27038 from viirya/partial-collect-accu.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-31 12:45:23 +09:00
Ajith f0fbbf014e [SPARK-30361][REST] Monitoring URL do not redact information about environment
UI and event logs redact sensitive information. But the monitoring URL, https://spark.apache.org/docs/latest/monitoring.html#rest-api , specifically /applications/[app-id]/environment does not, which is a security issue.

### What changes were proposed in this pull request?
REST api response is redacted before sending it

### Why are the changes needed?
If no redaction is done for rest API call, it can leak security information

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

### How was this patch tested?
Tested manually

Closes #27018 from ajithme/redactrest.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-30 09:10:31 -06:00
07ARB 5af77410bb [SPARK-30383][WEBUI] Remove meaning less tooltip from All pages
### What changes were proposed in this pull request?
Remove meaning less tooltip from All pages.

### Why are the changes needed?
If we can't come up with meaningful tooltips, then tooltips not require to add.

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

![67598045-351ab100-f78a-11e9-88cf-573e09d7c50e](https://user-images.githubusercontent.com/8948111/71558018-81c58580-2a74-11ea-9f38-dcaebd3f0bbf.png)

tooltips like highlight in above image got removed
### How was this patch tested?
Manual test.

Closes #27043 from 07ARB/SPARK-30383.

Authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-30 18:28:02 +09:00
Jungtaek Lim (HeartSaVioR) 8092d634ea [SPARK-30348][SPARK-27510][CORE][TEST] Fix flaky test failure on "MasterSuite.: Master should avoid ..."
### What changes were proposed in this pull request?

This patch fixes the flaky test failure on MasterSuite, "SPARK-27510: Master should avoid dead loop while launching executor failed in Worker".

The culprit of test failure was ironically the test ran too fast; the interval of `eventually` is by default "15 ms", but it took only "8 ms" from submitting driver to removing app from master.

```
19/12/23 15:45:06.533 dispatcher-event-loop-6 INFO Master: Registering worker localhost:9999 with 10 cores, 3.6 GiB RAM
19/12/23 15:45:06.534 dispatcher-event-loop-6 INFO Master: Driver submitted org.apache.spark.FakeClass
19/12/23 15:45:06.535 dispatcher-event-loop-6 INFO Master: Launching driver driver-20191223154506-0000 on worker 10001
19/12/23 15:45:06.536 dispatcher-event-loop-9 INFO Master: Registering app name
19/12/23 15:45:06.537 dispatcher-event-loop-9 INFO Master: Registered app name with ID app-20191223154506-0000
19/12/23 15:45:06.537 dispatcher-event-loop-9 INFO Master: Launching executor app-20191223154506-0000/0 on worker 10001
19/12/23 15:45:06.537 dispatcher-event-loop-10 INFO Master: Removing executor app-20191223154506-0000/0 because it is FAILED
...
19/12/23 15:45:06.542 dispatcher-event-loop-19 ERROR Master: Application name with ID app-20191223154506-0000 failed 10 times; removing it
```

Given the interval is already tiny, instead of lowering interval, the patch considers above case as well when verifying the status.

### Why are the changes needed?

We observed intermittent test failure in Jenkins build which should be fixed.
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115664/testReport/

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

No.

### How was this patch tested?

Modified UT.

Closes #27004 from HeartSaVioR/SPARK-30348.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-30 14:37:17 +08:00
yi.wu b5c35d68e4 [SPARK-27348][CORE] HeartbeatReceiver should remove lost executors from CoarseGrainedSchedulerBackend
### What changes were proposed in this pull request?

Remove it from `CoarseGrainedSchedulerBackend` when `HeartbeatReceiver` recognizes a lost executor.

### Why are the changes needed?

Currently, an application may hang if we don't remove a lost executor from `CoarseGrainedSchedulerBackend` as it may happens due to:

1) In `expireDeadHosts()`, `HeartbeatReceiver` calls `scheduler.executorLost()`;

2) Before `HeartbeatReceiver` calls `sc.killAndReplaceExecutor()`(which would mark the lost executor as "pendingToRemove") in a separate thread,  `CoarseGrainedSchedulerBackend` may begins to launch tasks on that executor without realizing it has been lost indeed.

3) If that lost executor doesn't shut down gracefully, `CoarseGrainedSchedulerBackend ` may never receive a disconnect event. As a result, tasks launched on that lost executor become orphans. While at the same time, driver just thinks that those tasks are still running and waits forever.

Removing the lost executor from `CoarseGrainedSchedulerBackend` would let `TaskSetManager` mark those tasks as failed which avoids app hang. Furthermore, it cleans up records in `executorDataMap`, which may never be removed in such case.

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

No

### How was this patch tested?

Updated existed tests.

Close #24350.

Closes #26980 from Ngone51/SPARK-27348.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-30 12:29:24 +08:00
Ajith 4257a94447 [SPARK-30360][UI] Avoid Redact classpath entries in History Server UI
Currently SPARK history server display the classpath entries in the Environment tab with classpaths redacted, this is because EventLog file has the entry values redacted while writing. But when same is seen from a running application UI, its seen that it is not redacted. Classpath entries redact is not needed and can be avoided

### What changes were proposed in this pull request?
Event logs will not redect the classpath entries

### Why are the changes needed?
Redact of classpath entries is not needed

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

### How was this patch tested?
Tested manually to verify on UI

Closes #27016 from ajithme/redactui.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-28 12:59:30 -06:00
yi.wu c35427f6b1 [SPARK-30355][CORE] Unify isExecutorActive between CoarseGrainedSchedulerBackend and DriverEndpoint
### What changes were proposed in this pull request?

Unify `DriverEndpoint. executorIsAlive()` and `CoarseGrainedSchedulerBackend .isExecutorActive()`.

### Why are the changes needed?

`DriverEndPoint` has method `executorIsAlive()` to check wether an executor is alive/active, while `CoarseGrainedSchedulerBackend` has method `isExecutorActive()` to do the same work. But, `isExecutorActive()` seems forget to consider `executorsPendingLossReason`. Unify these two methods makes behavior be consistent between `DriverEndPoint` and `CoarseGrainedSchedulerBackend` and make code more easier to maintain.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #27012 from Ngone51/unify-is-executor-alive.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 14:41:45 +08:00
Fu Chen 3584d84943 [MINOR][CORE] Quiet request executor remove message
### What changes were proposed in this pull request?

Settings to quiet for Class `ExecutorAllocationManager` that request message too verbose. otherwise, this class generates too many messages like
`INFO spark.ExecutorAllocationManager: Request to remove executorIds: 890`
 when we enabled DRA.

### Why are the changes needed?

Log level improvement.

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

No

### How was this patch tested?

Closes #26925 from cfmcgrady/quiet-request-executor-remove-message.

Authored-by: Fu Chen <cfmcgrady@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-26 09:59:41 -06:00
yi.wu 35506dced7 [SPARK-25855][CORE][FOLLOW-UP] Format config name to follow the other boolean conf naming convention
### What changes were proposed in this pull request?

Change config name from `spark.eventLog.allowErasureCoding` to `spark.eventLog.allowErasureCoding.enabled`.

### Why are the changes needed?

To follow the other boolean conf naming convention.

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

No, it's newly added in Spark 3.0.

### How was this patch tested?

Tested manually and pass Jenkins.

Closes #26998 from Ngone51/SPARK-25855-FOLLOWUP.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 19:24:58 +08:00
Liang-Chi Hsieh 0042ad575a [SPARK-30290][CORE] Count for merged block when fetching continuous blocks in batch
### What changes were proposed in this pull request?

We added shuffle block fetch optimization in SPARK-9853. In ShuffleBlockFetcherIterator, we merge single blocks into batch blocks. During merging, we should count merged blocks for `maxBlocksInFlightPerAddress`, not original single blocks.

### Why are the changes needed?

If `maxBlocksInFlightPerAddress` is specified, like set it to 1, it should mean one batch block, not one original single block. Otherwise, it will conflict with batch shuffle fetch.

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

No

### How was this patch tested?

Unit test.

Closes #26930 from viirya/respect-max-blocks-inflight.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 18:57:02 +08:00
Kazuaki Ishizaki f31d9a629b [MINOR][DOC][SQL][CORE] Fix typo in document and comments
### What changes were proposed in this pull request?

Fixed typo in `docs` directory and in other directories

1. Find typo in `docs` and apply fixes to files in all directories
2. Fix `the the` -> `the`

### Why are the changes needed?

Better readability of documents

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

No

### How was this patch tested?

No test needed

Closes #26976 from kiszk/typo_20191221.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-21 14:08:58 -08:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

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

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06:00
Kousuke Saruta 0945633844 [SPARK-29997][WEBUI][FOLLOWUP] Refactor code for job description of empty jobs
### What changes were proposed in this pull request?

Refactor the code brought by #26637 .
No more dummy StageInfo and its side-effects are needed at all.
This change also enable users to set job description to empty jobs though.

### Why are the changes needed?

The previous approach introduced dummy StageInfo and this causes side-effects.

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

Yes. Description set by user will be shown in the AllJobsPage.

![](https://user-images.githubusercontent.com/4736016/70788638-acf17900-1dd4-11ea-95f9-6d6739b24083.png)

### How was this patch tested?

Manual test and newly added unit test.

Closes #26703 from sarutak/fix-ui-for-empty-job2.

Lead-authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-18 10:27:31 -08:00
Liang-Chi Hsieh b2baaa2fcc [SPARK-30274][CORE] Avoid BytesToBytesMap lookup hang forever when holding keys reaching max capacity
### What changes were proposed in this pull request?

We should not append keys to BytesToBytesMap to be its max capacity.

### Why are the changes needed?

BytesToBytesMap.append allows to append keys until the number of keys reaches MAX_CAPACITY. But once the the pointer array in the map holds MAX_CAPACITY keys, next time call of lookup will hang forever.

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

No

### How was this patch tested?

Manually test by:
```java
Test
  public void testCapacity() {
    TestMemoryManager memoryManager2 =
            new TestMemoryManager(
                    new SparkConf()
                            .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true)
                            .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 25600 * 1024 * 1024L)
                            .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)
                            .set(package$.MODULE$.SHUFFLE_COMPRESS(), false));
    TaskMemoryManager taskMemoryManager2 = new TaskMemoryManager(memoryManager2, 0);
    final long pageSizeBytes = 8000000 + 8; // 8 bytes for end-of-page marker
    final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager2, 1024, pageSizeBytes);

    try {
      for (long i = 0; i < BytesToBytesMap.MAX_CAPACITY + 1; i++) {
        final long[] value = new long[]{i};
        boolean succeed = map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append(
                value,
                Platform.LONG_ARRAY_OFFSET,
                8,
                value,
                Platform.LONG_ARRAY_OFFSET,
                8);
      }
      map.free();
    } finally {
      map.free();
    }
  }
```

Once the map was appended to 536870912 keys (MAX_CAPACITY), the next lookup will hang.

Closes #26914 from viirya/fix-bytemap2.

Authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-17 11:37:05 -08:00
“attilapiros” cdc8fc6233 [SPARK-30235][CORE] Switching off host local disk reading of shuffle blocks in case of useOldFetchProtocol
### What changes were proposed in this pull request?

When `spark.shuffle.useOldFetchProtocol` is enabled then switching off the direct disk reading of host-local shuffle blocks and falling back to remote block fetching (and this way avoiding the `GetLocalDirsForExecutors` block transfer message which is introduced from Spark 3.0.0).

### Why are the changes needed?

In `[SPARK-27651][Core] Avoid the network when shuffle blocks are fetched from the same host` a new block transfer message is introduced, `GetLocalDirsForExecutors`. This new message could be sent to the external shuffle service and as it is not supported by the previous version of external shuffle service it should be avoided when `spark.shuffle.useOldFetchProtocol` is true.

In the migration guide I changed the exception type as `org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Decoder#fromByteBuffer`
throws a IllegalArgumentException with the given text and uses the message type which is just a simple number (byte). I have checked and this is true for version 2.4.4 too.

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

### How was this patch tested?

This specific case (considering one extra boolean to switch off host local disk reading feature) is not tested but existing tests were run.

Closes #26869 from attilapiros/SPARK-30235.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-17 10:32:15 -08:00
Xingbo Jiang 1c714befd8 [SPARK-25100][TEST][FOLLOWUP] Refactor test cases in FileSuite and KryoSerializerSuite
### What changes were proposed in this pull request?

Refactor test cases added by https://github.com/apache/spark/pull/26714, to improve code compactness.

### How was this patch tested?

Tested locally.

Closes #26916 from jiangxb1987/SPARK-25100.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-16 21:11:15 -08:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

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

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
shahid dd217e10fc [SPARK-25392][CORE][WEBUI] Prevent error page when accessing pools page from history server
### What changes were proposed in this pull request?

### Why are the changes needed?

Currently from history server, we will not able to access the pool info, as we aren't writing pool information to the event log other than pool name. Already spark is hiding pool table when accessing from history server. But from the pool column in the stage table will redirect to the pools table, and that will throw error when accessing the pools page. To prevent error page, we need to hide the pool column also in the stage table

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

No

### How was this patch tested?
Manual test

Before change:
![Screenshot 2019-11-21 at 6 49 40 AM](https://user-images.githubusercontent.com/23054875/69293868-219b2280-0c30-11ea-9b9a-17140d024d3a.png)
![Screenshot 2019-11-21 at 6 48 51 AM](https://user-images.githubusercontent.com/23054875/69293834-147e3380-0c30-11ea-9dec-d5f67665486d.png)

After change:
![Screenshot 2019-11-21 at 7 29 01 AM](https://user-images.githubusercontent.com/23054875/69293991-9cfcd400-0c30-11ea-98a0-7a6268a4e5ab.png)

Closes #26616 from shahidki31/poolHistory.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-16 15:02:34 -08:00
turbofei 5954311739 [SPARK-29043][CORE] Improve the concurrent performance of History Server
Even we set spark.history.fs.numReplayThreads to a large number, such as 30.
The history server still replays logs slowly.
We found that, if there is a straggler in a batch of replay tasks, all the other threads will wait for this
straggler.

In this PR, we create processing to save the logs which are being replayed.
So that the replay tasks can execute Asynchronously.

It can accelerate the speed to replay logs  for history server.

No.

UT.

Closes #25797 from turboFei/SPARK-29043.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-16 14:45:27 -08:00
Marcelo Vanzin a9fbd31030 [SPARK-30240][CORE] Support HTTP redirects directly to a proxy server
### What changes were proposed in this pull request?

The PR adds a new config option to configure an address for the
proxy server, and a new handler that intercepts redirects and replaces
the URL with one pointing at the proxy server. This is needed on top
of the "proxy base path" support because redirects use full URLs, not
just absolute paths from the server's root.

### Why are the changes needed?

Spark's web UI has support for generating links to paths with a
prefix, to support a proxy server, but those do not apply when
the UI is responding with redirects. In that case, Spark is sending
its own URL back to the client, and if it's behind a dumb proxy
server that doesn't do rewriting (like when using stunnel for HTTPS
support) then the client will see the wrong URL and may fail.

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

Yes. It's a new UI option.

### How was this patch tested?

Tested with added unit test, with Spark behind stunnel, and in a
more complicated app using a different HTTPS proxy.

Closes #26873 from vanzin/SPARK-30240.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 17:39:06 -08:00
xiaodeshan fb2f5a4906 [SPARK-25100][CORE] Register TaskCommitMessage to KyroSerializer
## What changes were proposed in this pull request?

Fix the bug when invoking saveAsNewAPIHadoopDataset to store data, the job will fail because the class TaskCommitMessage hasn't be registered if serializer is KryoSerializer and spark.kryo.registrationRequired is true

## How was this patch tested?

UT

Closes #26714 from deshanxiao/SPARK-25100.

Authored-by: xiaodeshan <xiaodeshan@xiaomi.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 17:15:30 -08:00
Sean Owen 46e950bea8 [SPARK-30263][CORE] Don't log potentially sensitive value of non-Spark properties ignored in spark-submit
### What changes were proposed in this pull request?

The value of non-Spark config properties ignored in spark-submit is no longer logged.

### Why are the changes needed?

The value isn't really needed in the logs, and could contain potentially sensitive info. While we can redact the values selectively too, I figured it's more robust to just not log them at all here, as the values aren't important in this log statement.

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

Other than the change to logging above, no.

### How was this patch tested?

Existing tests

Closes #26893 from srowen/SPARK-30263.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 13:13:54 -08:00
Kousuke Saruta 61ebc81186 [SPARK-30167][REPL] Log4j configuration for REPL can't override the root logger properly
### What changes were proposed in this pull request?

In the current implementation of `SparkShellLoggingFilter`, if the log level of the root logger and the log level of a message are different, whether a message should logged is decided based on log4j's configuration but whether the message should be output to the REPL's console is not cared.
So, if the log level of the root logger is `DEBUG`, the log level of REPL's logger is `WARN` and the log level of a message is `INFO`, the message will output to the REPL's console even though `INFO < WARN`.
https://github.com/apache/spark/pull/26798/files#diff-bfd5810d8aa78ad90150e806d830bb78L237

The ideal behavior should be like as follows and implemented them in this change.

1. If the log level of a message is greater than or equal to the log level of the root logger, the message should be logged but whether the message is output to the REPL's console should be decided based on whether the log level of the message is greater than or equal to the log level of the REPL's logger.

2. If a log level or custom appenders are explicitly defined for a category, whether a log message via the logger corresponding to the category is logged and output to the REPL's console should be decided baed on the log level of the category.
We can confirm whether a log level or appenders are explicitly set to a logger for a category by `Logger#getLevel` and `Logger#getAllAppenders.hasMoreElements`.

### Why are the changes needed?

This is a bug breaking a compatibility.

#9816 enabled REPL's log4j configuration to override root logger but #23675 seems to have broken the feature.
You can see one example when you modifies the default log4j configuration like as follows.
```
# Change the log level for rootCategory to DEBUG
log4j.rootCategory=DEBUG, console

...
# The log level for repl.Main remains WARN
log4j.logger.org.apache.spark.repl.Main=WARN
```
If you launch REPL with the configuration, INFO level logs appear even though the log level for REPL is WARN.
```
・・・

19/12/08 23:31:38 INFO Utils: Successfully started service 'sparkDriver' on port 33083.
19/12/08 23:31:38 INFO SparkEnv: Registering MapOutputTracker
19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMaster
19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information
19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up
19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMasterHeartbeat

・・・
```
Before #23675 was applied, those INFO level logs are not shown with the same log4j.properties.

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

Yes. The logging behavior for REPL is fixed.

### How was this patch tested?

Manual test and newly added unit test.

Closes #26798 from sarutak/fix-spark-shell-loglevel.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-13 14:30:11 -08:00
sharan.gk ec26dde36b [SPARK-29455][WEBUI] Improve tooltip information for Stages
### What changes were proposed in this pull request?
Adding tooltip to Stages tab for better usability.

### Why are the changes needed?
There are a few common points of confusion in the UI that could be clarified with tooltips. We
should add tooltips to explain.

### Does this PR introduce any user-facing change?
Yes
![image](https://user-images.githubusercontent.com/29914590/70693889-5a389400-1ce4-11ea-91bb-ee1e997a5c35.png)

### How was this patch tested?
Manual

Closes #26859 from sharangk/tooltip1.

Authored-by: sharan.gk <sharan.gk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-13 11:35:00 -08:00
07ARB ce61ee8941 [SPARK-30126][CORE] support space in file path and name for addFile and addJar function
### What changes were proposed in this pull request?
sparkContext.addFile and sparkContext.addJar fails when file path contains spaces

### Why are the changes needed?
When uploading a file to the spark context via the addFile and addJar function, an exception is thrown when file path contains a space character. Escaping the space with %20 or
or + doesn't change the result.

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

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

Closes #26773 from 07ARB/SPARK-30126.

Authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-12 20:30:47 +08:00
Liang-Chi Hsieh b4aeaf906f [SPARK-30198][CORE] BytesToBytesMap does not grow internal long array as expected
### What changes were proposed in this pull request?

This patch changes the condition to check if BytesToBytesMap should grow up its internal array. Specifically, it changes to compare by the capacity of the array, instead of its size.

### Why are the changes needed?

One Spark job on our cluster hangs forever at BytesToBytesMap.safeLookup. After inspecting, the long array size is 536870912.

Currently in BytesToBytesMap.append, we only grow the internal array if the size of the array is less than its MAX_CAPACITY that is 536870912. So in above case, the array can not be grown up, and safeLookup can not find an empty slot forever.

But it is wrong because we use two array entries per key, so the array size is twice the capacity. We should compare the current capacity of the array, instead of its size.

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

No

### How was this patch tested?

This issue only happens when loading big number of values into BytesToBytesMap, so it is hard to do unit test. This is tested manually with internal Spark job.

Closes #26828 from viirya/fix-bytemap.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 14:58:21 -08:00
Sean Owen 33f53cb2d5 [SPARK-30195][SQL][CORE][ML] Change some function, import definitions to work with stricter compiler in Scala 2.13
### What changes were proposed in this pull request?

See https://issues.apache.org/jira/browse/SPARK-30195 for the background; I won't repeat it here. This is sort of a grab-bag of related issues.

### Why are the changes needed?

To cross-compile with Scala 2.13 later.

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

No.

### How was this patch tested?

Existing tests for 2.12. I've been manually checking that this actually resolves the compile problems in 2.13 separately.

Closes #26826 from srowen/SPARK-30195.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-11 12:33:58 -08:00
Pavithra Ramachandran d46c03c3d3 [SPARK-29460][WEBUI] Add tooltip for Jobs page
### What changes were proposed in this pull request?
Adding tooltip for jobs tab column - Job Id (Job Group), Description ,Submitted, Duration, Stages, Tasks

Before:
![Screenshot from 2019-11-04 11-31-02](https://user-images.githubusercontent.com/51401130/68102467-e8a54300-fef8-11e9-9f9e-48dd1b393ac8.png)

After:
![Screenshot from 2019-11-04 11-30-53](https://user-images.githubusercontent.com/51401130/68102478-f3f86e80-fef8-11e9-921a-357678229cb4.png)

### Why are the changes needed?
Jobs tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience.

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

### How was this patch tested?
Manual

Closes #26384 from PavithraRamachandran/jobTab_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-11 09:39:39 -06:00
root1 d7843dde0f [SPARK-29152][CORE] Executor Plugin shutdown when dynamic allocation is enabled
### What changes were proposed in this pull request?
Added `shutdownHook` for shutdown method of executor plugin. This will ensure that shutdown method will be called always.

### Why are the changes needed?
Whenever executors are not going down gracefully, i.e getting killed due to idle time or getting killed forcefully, shutdown method of executors plugin is not getting called. Shutdown method can be used to release any resources that plugin has acquired during its initialisation. So its important to make sure that every time a executor goes down shutdown method of plugin gets called.

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

### How was this patch tested?

Tested Manually.

Closes #26810 from iRakson/Executor_Plugin.

Authored-by: root1 <raksonrakesh@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-12-10 14:23:51 -08:00
Yuchen Huo ad238a2238 [SPARK-29976][CORE] Trigger speculation for stages with too few tasks
### What changes were proposed in this pull request?
This PR add an optional spark conf for speculation to allow speculative runs for stages where there are only a few tasks.
```
spark.speculation.task.duration.threshold
```

If provided, tasks would be speculatively run if the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold.

### Why are the changes needed?
This change helps avoid scenarios where there is single executor that could hang forever due to disk issue and we unfortunately assigned the single task in a TaskSet to that executor and cause the whole job to hang forever.

### Does this PR introduce any user-facing change?
yes. If the new config `spark.speculation.task.duration.threshold` is provided and the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold, then speculative tasks would be submitted for the running tasks in the TaskSet.

### How was this patch tested?
Unit tests are added to TaskSetManagerSuite.

Closes #26614 from yuchenhuo/SPARK-29976.

Authored-by: Yuchen Huo <yuchen.huo@databricks.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-12-10 14:43:26 -06:00
Luca Canali 729f43f499 [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system
## What changes were proposed in this pull request?

This PR proposes to add instrumentation of memory usage via the Spark Dropwizard/Codahale metrics system. Memory usage metrics are available via the Executor metrics, recently implemented as detailed in https://issues.apache.org/jira/browse/SPARK-23206.
Additional notes: This takes advantage of the metrics poller introduced in #23767.

## Why are the changes needed?
Executor metrics bring have many useful insights on memory usage, in particular on the usage of storage memory and executor memory. This is useful for troubleshooting. Having the information in the metrics systems allows to add those metrics to Spark performance dashboards and study memory usage as a function of time, as in the example graph https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG

## Does this PR introduce any user-facing change?
Adds `ExecutorMetrics` source to publish executor metrics via the Dropwizard metrics system. Details of the available metrics in docs/monitoring.md
Adds configuration parameter `spark.metrics.executormetrics.source.enabled`

## How was this patch tested?

Tested on YARN cluster and with an existing setup for a Spark dashboard based on InfluxDB and Grafana.

Closes #24132 from LucaCanali/memoryMetricsSource.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-12-09 08:55:30 -06:00
Sean Owen ebd83a544e [SPARK-30009][CORE][SQL][FOLLOWUP] Remove OrderingUtil and Utils.nanSafeCompare{Doubles,Floats} and use java.lang.{Double,Float}.compare directly
### What changes were proposed in this pull request?

Follow up on https://github.com/apache/spark/pull/26654#discussion_r353826162
Instead of OrderingUtil or Utils.nanSafeCompare{Doubles,Floats}, just use java.lang.{Double,Float}.compare directly. All work identically w.r.t. NaN when used to `compare`.

### Why are the changes needed?

Simplification of the previous change, which existed to support Scala 2.13 migration.

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

No.

### How was this patch tested?

Existing tests

Closes #26761 from srowen/SPARK-30009.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-05 11:27:25 +08:00
Luca Canali 60f20e5ea2 [SPARK-30060][CORE] Rename metrics enable/disable configs
### What changes were proposed in this pull request?
This proposes to introduce a naming convention for Spark metrics configuration parameters used to enable/disable metrics source reporting using the Dropwizard metrics library:   `spark.metrics.sourceNameCamelCase.enabled` and update 2 parameters to use this naming convention.

### Why are the changes needed?
Currently Spark has a few parameters to enable/disable metrics reporting. Their naming pattern is not uniform and this can create confusion.  Currently we have:
`spark.metrics.static.sources.enabled`
`spark.app.status.metrics.enabled`
`spark.sql.streaming.metricsEnabled`

### Does this PR introduce any user-facing change?
Update parameters for enabling/disabling metrics reporting new in Spark 3.0: `spark.metrics.static.sources.enabled` -> `spark.metrics.staticSources.enabled`, `spark.app.status.metrics.enabled`  -> `spark.metrics.appStatusSource.enabled`.
Note: `spark.sql.streaming.metricsEnabled` is left unchanged as it is already in use in Spark 2.x.

### How was this patch tested?
Manually tested

Closes #26692 from LucaCanali/uniformNamingMetricsEnableParameters.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-03 14:31:06 -08:00
Sean Owen 4193d2f4cc [SPARK-30012][CORE][SQL] Change classes extending scala collection classes to work with 2.13
### What changes were proposed in this pull request?

Move some classes extending Scala collections into parallel source trees, to support 2.13; other minor collection-related modifications.

Modify some classes extending Scala collections to work with 2.13 as well as 2.12. In many cases, this means introducing parallel source trees, as the type hierarchy changed in ways that one class can't support both.

### Why are the changes needed?

To support building for Scala 2.13 in the future.

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

There should be no behavior change.

### How was this patch tested?

Existing tests. Note that the 2.13 changes are not tested by the PR builder, of course. They compile in 2.13 but can't even be tested locally. Later, once the project can be compiled for 2.13, thus tested, it's possible the 2.13 implementations will need updates.

Closes #26728 from srowen/SPARK-30012.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-03 08:59:43 -08:00
Yuanjian Li 169415ffac [SPARK-30025][CORE] Continuous shuffle block fetching should be disabled by default when the old fetch protocol is used
### What changes were proposed in this pull request?
Disable continuous shuffle block fetching when the old fetch protocol in use.

### Why are the changes needed?
The new feature of continuous shuffle block fetching depends on the latest version of the shuffle fetch protocol. We should keep this constraint in `BlockStoreShuffleReader.fetchContinuousBlocksInBatch`.

### Does this PR introduce any user-facing change?
Users will not get the exception related to continuous shuffle block fetching when old version of the external shuffle service is used.

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

Closes #26663 from xuanyuanking/SPARK-30025.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-02 15:59:12 +08:00
shahid b182ed83f6 [SPARK-29724][SPARK-29726][WEBUI][SQL] Support JDBC/ODBC tab for HistoryServer WebUI
### What changes were proposed in this pull request?

 Support JDBC/ODBC tab for HistoryServer WebUI. Currently from Historyserver we can't access the JDBC/ODBC tab for thrift server applications. In this PR, I am doing 2 main changes
1. Refactor existing thrift server listener to support kvstore
2. Add history server plugin for thrift server listener and tab.

### Why are the changes needed?
Users can access Thriftserver tab from History server for both running and finished applications,

### Does this PR introduce any user-facing change?
Support for JDBC/ODBC tab  for the WEBUI from History server

### How was this patch tested?
Add UT and Manual tests
1. Start Thriftserver and Historyserver
```
sbin/stop-thriftserver.sh
sbin/stop-historyserver.sh
sbin/start-thriftserver.sh
sbin/start-historyserver.sh
```
2. Launch beeline
`bin/beeline -u jdbc:hive2://localhost:10000`

3. Run queries

Go to the JDBC/ODBC page of the WebUI from History server

![image](https://user-images.githubusercontent.com/23054875/68365501-cf013700-0156-11ea-84b4-fda8008c92c4.png)

Closes #26378 from shahidki31/ThriftKVStore.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-11-29 19:44:31 -08:00
wuyi d075b3344e [SPARK-28366][CORE][FOLLOW-UP] Improve the conf IO_WARNING_LARGEFILETHRESHOLD
### What changes were proposed in this pull request?

Improve conf `IO_WARNING_LARGEFILETHRESHOLD` (a.k.a `spark.io.warning.largeFileThreshold`):

* reword documentation

* change type from `long` to `bytes`

### Why are the changes needed?

Improvements according to https://github.com/apache/spark/pull/25134#discussion_r350570804 & https://github.com/apache/spark/pull/25134#discussion_r350570917.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #26691 from Ngone51/SPARK-28366-followup.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-27 20:34:22 +08:00
Kousuke Saruta 08e2a39df2 [SPARK-29997][WEBUI] Show job name for empty jobs in WebUI
### What changes were proposed in this pull request?

In current implementation, job name for empty jobs is not shown so I've made change to show it.

### Why are the changes needed?

To make debug easier.

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

Yes. Before applying my change, the `Job Page` will show as follows as the result of submitting a job which contains no partitions.

![fix-ui-for-empty-job-before](https://user-images.githubusercontent.com/4736016/69410847-33bfb280-0d4f-11ea-9878-d67638cbe4cb.png)

After applying my change, the page will show a display like a following screenshot.

![fix-ui-for-empty-job-after](https://user-images.githubusercontent.com/4736016/69411021-86996a00-0d4f-11ea-8dea-bb8456159d18.png)

### How was this patch tested?

Manual test.

Closes #26637 from sarutak/fix-ui-for-empty-job.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-26 19:38:46 -08:00
“attilapiros” fd2bf55aba [SPARK-27651][CORE] Avoid the network when shuffle blocks are fetched from the same host
## What changes were proposed in this pull request?

Before this PR `ShuffleBlockFetcherIterator` was partitioning the block fetches into two distinct sets: local reads and remote fetches. Within this PR (when the feature is enabled by "spark.shuffle.readHostLocalDisk.enabled") a new category is introduced: host-local reads. They are shuffle block fetches where although the block manager is different they are running on the same host along with the requester.

Moreover to get the local directories of the other executors/block managers a new RPC message is introduced `GetLocalDirs` which is sent the the block manager master where it is answered as `BlockManagerLocalDirs`. In `BlockManagerMasterEndpoint` for answering this request the `localDirs` is extracted from the `BlockManagerInfo` and stored separately in a hash map called `executorIdLocalDirs`. Because the earlier used `blockManagerInfo` contains data for the alive block managers (see `org.apache.spark.storage.BlockManagerMasterEndpoint#removeBlockManager`).

Now `executorIdLocalDirs` knows all the local dirs up to the application start (like the external shuffle service does) so in case of an RDD recalculation both host-local shuffle blocks and disk persisted RDD blocks on the same host can be served by reading the files behind the blocks directly.

## How was this patch tested?

### Unit tests

`ExternalShuffleServiceSuite`:
- "SPARK-27651: host local disk reading avoids external shuffle service on the same node"

`ShuffleBlockFetcherIteratorSuite`:
- "successful 3 local reads + 4 host local reads + 2 remote reads"

And with extending existing suites where shuffle metrics was tested.

### Manual tests

Running Spark on YARN in a 4 nodes cluster with 6 executors and having 12 shuffle blocks.

```
$ grep host-local experiment.log
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_2_1, shuffle_0_6_1
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 38 ms
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_0_0, shuffle_0_8_0
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 35 ms
```

Closes #25299 from attilapiros/SPARK-27651.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-26 11:02:25 -08:00
Sean Owen 29018025ba [SPARK-30009][CORE][SQL] Support different floating-point Ordering for Scala 2.12 / 2.13
### What changes were proposed in this pull request?

Make separate source trees for Scala 2.12/2.13 in order to accommodate mutually-incompatible support for Ordering of double, float.

Note: This isn't the last change that will need a split source tree for 2.13. But this particular change could go several ways:

- (Split source tree)
- Inline the Scala 2.12 implementation
- Reflection

For this change alone any are possible, and splitting the source tree is a bit overkill. But if it will be necessary for other JIRAs (see umbrella SPARK-25075), then it might be the easiest way to implement this.

### Why are the changes needed?

Scala 2.13 split Ordering.Double into Ordering.Double.TotalOrdering and Ordering.Double.IeeeOrdering. Neither can be used in a single build that supports 2.12 and 2.13.

TotalOrdering works like java.lang.Double.compare. IeeeOrdering works like Scala 2.12 Ordering.Double. They differ in how NaN is handled - compares always above other values? or always compares as 'false'? In theory they have different uses: TotalOrdering is important if floating-point values are sorted. IeeeOrdering behaves like 2.12 and JVM comparison operators.

I chose TotalOrdering as I think we care more about stable sorting, and because elsewhere we rely on java.lang comparisons. It is also possible to support with two methods.

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

Pending tests, will see if it obviously affects any sort order. We need to see if it changes NaN sort order.

### How was this patch tested?

Existing tests so far.

Closes #26654 from srowen/SPARK-30009.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-26 08:25:53 -08:00
wuyi 7b1b60c758 [SPARK-28574][CORE][FOLLOW-UP] Several minor improvements for event queue capacity config
### What changes were proposed in this pull request?

* Replace hard-coded conf `spark.scheduler.listenerbus.eventqueue` with a constant variable(`LISTENER_BUS_EVENT_QUEUE_PREFIX `) defined in `config/package.scala`.

* Update documentation for `spark.scheduler.listenerbus.eventqueue.capacity` in both `config/package.scala` and `docs/configuration.md`.

### Why are the changes needed?

* Better code maintainability

* Better user guidance of the conf

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

No behavior changes but user will see the updated document.

### How was this patch tested?

Pass Jenkins.

Closes #26676 from Ngone51/SPARK-28574-followup.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-26 08:20:26 -08:00
Dongjoon Hyun 9b9d130f15 [SPARK-30030][BUILD][FOLLOWUP] Remove unused org.apache.commons.lang
### What changes were proposed in this pull request?

This PR aims to remove the unused test dependency `commons-lang:commons-lang` from `core` module.

### Why are the changes needed?

SPARK-30030 already removed all usage of `Apache Commons Lang2` in `core`.

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

No.

### How was this patch tested?

Pass the Jenkins.

Closes #26673 from dongjoon-hyun/SPARK-30030-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-26 20:55:02 +09:00
wuyi 780555bf60 [MINOR][CORE] Make EventLogger codec be consistent between EventLogFileWriter and SparkContext
### What changes were proposed in this pull request?

Use the same function (`codecName(conf: SparkConf)`) between `EventLogFileWriter` and `SparkContext` to get the consistent codec name for EventLogger.

### Why are the changes needed?

#24921 added a new conf for EventLogger's compression codec. We should reflect this change into `SparkContext` as well. Though I didn't find any places that `SparkContext.eventLogCodec` really takes an effect, I think it'd be better to have it as a right value.

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

No.

### How was this patch tested?

Pass Jenkins.

Closes #26665 from Ngone51/consistent-eventLogCodec.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-26 12:54:34 +08:00
Dongjoon Hyun 38240a74dc [SPARK-30030][INFRA] Use RegexChecker instead of TokenChecker to check org.apache.commons.lang.
### What changes were proposed in this pull request?

This PR replace `TokenChecker` with `RegexChecker` in `scalastyle` and fixes the missed instances.

### Why are the changes needed?

This will remove the old `comons-lang2` dependency from `core` module

**BEFORE**
```
$ dev/scalastyle
Scalastyle checks failed at following occurrences:
[error] /Users/dongjoon/PRS/SPARK-SerializationUtils/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:23:7: Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead
[error]     of Commons Lang 2 (package org.apache.commons.lang.*)
[error] Total time: 23 s, completed Nov 25, 2019 11:47:44 AM
```

**AFTER**
```
$ dev/scalastyle
Scalastyle checks passed.
```

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

No.

### How was this patch tested?

Pass the GitHub Action linter.

Closes #26666 from dongjoon-hyun/SPARK-29081-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-25 12:03:15 -08:00
Dongjoon Hyun 1466863cee [SPARK-30015][BUILD] Move hive-storage-api dependency from hive-2.3 to sql/core
# What changes were proposed in this pull request?

This PR aims to relocate the following internal dependencies to compile `sql/core` without `-Phive-2.3` profile.

1. Move the `hive-storage-api` to `sql/core` which is using `hive-storage-api` really.

**BEFORE (sql/core compilation)**
```
$ ./build/mvn -DskipTests --pl sql/core --am compile
...
[ERROR] [Error] /Users/dongjoon/APACHE/spark/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala:21: object hive is not a member of package org.apache.hadoop
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
```
**AFTER (sql/core compilation)**
```
$ ./build/mvn -DskipTests --pl sql/core --am compile
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time:  02:04 min
[INFO] Finished at: 2019-11-25T00:20:11-08:00
[INFO] ------------------------------------------------------------------------
```

2. For (1), add `commons-lang:commons-lang` test dependency to `spark-core` module to manage the dependency explicitly. Without this, `core` module fails to build the test classes.

```
$ ./build/mvn -DskipTests --pl core --am package -Phadoop-3.2
...
[INFO] --- scala-maven-plugin:4.3.0:testCompile (scala-test-compile-first)  spark-core_2.12 ---
[INFO] Using incremental compilation using Mixed compile order
[INFO] Compiler bridge file: /Users/dongjoon/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.12-1.3.1-bin_2.12.10__52.0-1.3.1_20191012T045515.jar
[INFO] Compiling 271 Scala sources and 26 Java sources to /spark/core/target/scala-2.12/test-classes ...
[ERROR] [Error] /spark/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:23: object lang is not a member of package org.apache.commons
[ERROR] [Error] /spark/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:49: not found: value SerializationUtils
[ERROR] two errors found
```

**BEFORE (commons-lang:commons-lang)**
The following is the previous `core` module's `commons-lang:commons-lang` dependency.

1. **branch-2.4**
```
$ mvn dependency:tree -Dincludes=commons-lang:commons-lang
[INFO] --- maven-dependency-plugin:3.0.2:tree (default-cli)  spark-core_2.11 ---
[INFO] org.apache.spark:spark-core_2.11🫙2.4.5-SNAPSHOT
[INFO] \- org.spark-project.hive:hive-exec:jar:1.2.1.spark2:provided
[INFO]    \- commons-lang:commons-lang:jar:2.6:compile
```

2. **v3.0.0-preview (-Phadoop-3.2)**
```
$ mvn dependency:tree -Dincludes=commons-lang:commons-lang -Phadoop-3.2
[INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli)  spark-core_2.12 ---
[INFO] org.apache.spark:spark-core_2.12🫙3.0.0-preview
[INFO] \- org.apache.hive:hive-storage-api:jar:2.6.0:compile
[INFO]    \- commons-lang:commons-lang:jar:2.6:compile
```

3. **v3.0.0-preview(default)**
```
$ mvn dependency:tree -Dincludes=commons-lang:commons-lang
[INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli)  spark-core_2.12 ---
[INFO] org.apache.spark:spark-core_2.12🫙3.0.0-preview
[INFO] \- org.apache.hadoop:hadoop-client:jar:2.7.4:compile
[INFO]    \- org.apache.hadoop:hadoop-common:jar:2.7.4:compile
[INFO]       \- commons-lang:commons-lang:jar:2.6:compile
```

**AFTER (commons-lang:commons-lang)**
```
$ mvn dependency:tree -Dincludes=commons-lang:commons-lang
[INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli)  spark-core_2.12 ---
[INFO] org.apache.spark:spark-core_2.12🫙3.0.0-SNAPSHOT
[INFO] \- commons-lang:commons-lang:jar:2.6:test
```

Since we wanted to verify that this PR doesn't change `hive-1.2` profile, we merged
[SPARK-30005 Update `test-dependencies.sh` to check `hive-1.2/2.3` profile](a1706e2fa7) before this PR.

### Why are the changes needed?

- Apache Spark 2.4's `sql/core` is using `Apache ORC (nohive)` jars including shaded `hive-storage-api` to access ORC data sources.

- Apache Spark 3.0's `sql/core` is using `Apache Hive` jars directly. Previously, `-Phadoop-3.2` hid this `hive-storage-api` dependency. Now, we are using `-Phive-2.3` instead. As I mentioned [previously](https://github.com/apache/spark/pull/26619#issuecomment-556926064), this PR is required to compile `sql/core` module without `-Phive-2.3`.

- For `sql/hive` and `sql/hive-thriftserver`, it's natural that we need `-Phive-1.2` or `-Phive-2.3`.

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

No.

### How was this patch tested?

This will pass the Jenkins (with the dependency check and unit tests).

We need to check manually with `./build/mvn -DskipTests --pl sql/core --am compile`.

This closes #26657 .

Closes #26658 from dongjoon-hyun/SPARK-30015.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-25 10:54:14 -08:00
shahid bec2068ae8 [SPARK-26260][CORE] For disk store tasks summary table should show only successful tasks summary
…sks metrics for disk store

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

After https://github.com/apache/spark/pull/23088 task Summary table in the stage page shows successful tasks metrics for lnMemory store. In this PR, it added for disk store also.

### Why are the changes needed?

Now both InMemory and disk store will be consistent in showing the task summary table in the UI, if there are non successful tasks

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

no
### How was this patch tested?

Added UT. Manually verified

Test steps:
1. add the config in spark-defaults.conf -> **spark.history.store.path /tmp/store**
2. sbin/start-hitoryserver
3. bin/spark-shell
4. `sc.parallelize(1 to 1000, 2).map(x => throw new Exception("fail")).count`

![Screenshot 2019-11-14 at 3 51 39 AM](https://user-images.githubusercontent.com/23054875/68809546-268d2e80-0692-11ea-8b2c-bee767478135.png)

Closes #26508 from shahidki31/task.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-25 10:04:25 -08:00
Thomas Graves 2d5de25a99 [SPARK-29415][CORE] Stage Level Sched: Add base ResourceProfile and Request classes
### What changes were proposed in this pull request?

This PR is adding the base classes needed for Stage level scheduling. Its adding a ResourceProfile and the executor and task resource request classes.  These are made private for now until we get all the parts implemented, at which point this will become public interfaces.  I am adding them first as all the other subtasks for this feature require these classes.  If people have better ideas on breaking this feature up please let me know.

See https://issues.apache.org/jira/browse/SPARK-29415 for more detailed design.

### Why are the changes needed?

New API for stage level scheduling.  Its easier to add these first because the other jira for this features will all use them.

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

Yes adds API to create a ResourceProfile with executor/task resources, see the spip jira https://issues.apache.org/jira/browse/SPARK-27495

Example of the api:
val rp = new ResourceProfile()
rp.require(new ExecutorResourceRequest("cores", 2))
rp.require(new ExecutorResourceRequest("gpu", 1, Some("/opt/gpuScripts/getGpus")))
rp.require(new TaskResourceRequest("gpu", 1))

### How was this patch tested?

Tested using Unit tests added with this PR.

Closes #26284 from tgravescs/SPARK-29415.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-11-25 09:36:39 -06:00
wuyi 456cfe6e46 [SPARK-29939][CORE] Add spark.shuffle.mapStatus.compression.codec conf
### What changes were proposed in this pull request?

Add a new conf named `spark.shuffle.mapStatus.compression.codec` for user to decide which codec should be used(default by `zstd`) for `MapStatus` compression.

### Why are the changes needed?

We already have this functionality for `broadcast`/`rdd`/`shuffle`/`shuflleSpill`,
so it might be better to have the same functionality for `MapStatus` as well.

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

Yes, user now could use `spark.shuffle.mapStatus.compression.codec` to decide which codec should be used during `MapStatus` compression.
### How was this patch tested?

N/A

Closes #26611 from Ngone51/SPARK-29939.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-24 21:21:19 -08:00
Prakhar Jain 6898be9f02 [SPARK-29681][WEBUI] Support column sorting in Environment tab
### What changes were proposed in this pull request?
Add extra classnames to table headers in EnvironmentPage tables in Spark UI.

### Why are the changes needed?
SparkUI uses sorttable.js to provide the sort functionality in different tables. This library tries to guess the datatype of each column during initialization phase - numeric/alphanumeric etc and uses it to sort the columns whenever user clicks on a column. That way it guesses incorrect data type for environment tab.

sorttable.js has way to hint the datatype of table columns explicitly. This is done by passing custom HTML class attribute.

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

### How was this patch tested?
Manually tested sorting in tables in Environment tab in Spark UI.

![Annotation 2019-11-22 154058](https://user-images.githubusercontent.com/2551496/69417432-a8d6bc00-0d3e-11ea-865b-f8017976c6f4.png)
![Annotation 2019-11-22 153600](https://user-images.githubusercontent.com/2551496/69417433-a8d6bc00-0d3e-11ea-9a75-8e1f4d66107e.png)
![Annotation 2019-11-22 153841](https://user-images.githubusercontent.com/2551496/69417435-a96f5280-0d3e-11ea-85f6-9f61b015e161.png)

Closes #26638 from prakharjain09/SPARK-29681-SPARK-UI-SORT.

Authored-by: Prakhar Jain <prakjai@microsoft.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-23 18:09:02 -08:00
Kousuke Saruta 6cd6d5f57e [SPARK-29970][WEBUI] Preserver open/close state of Timelineview
### What changes were proposed in this pull request?

Fix a bug related to Timelineview that does not preserve open/close state properly.

### Why are the changes needed?

To preserve open/close state is originally intended but it doesn't work.

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

Yes. open/close state for Timeineview is to be preserved so if you open Timelineview in Stage page and go to another page, and then go back to Stage page, Timelineview should keep open.

### How was this patch tested?

Manual test.

Closes #26607 from sarutak/fix-timeline-view-state.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-23 16:16:24 -08:00
gengjiaan 85c004d5b0 [SPARK-29885][PYTHON][CORE] Improve the exception message when reading the daemon port
### What changes were proposed in this pull request?
In production environment, my PySpark application occurs an exception and it's message as below:
```
19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
org.apache.spark.SparkException: No port number in pyspark.daemon's stdout
	at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:204)
	at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122)
	at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95)
	at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117)
	at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108)
	at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337)
	at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335)
	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182)
	at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
	at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
	at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:121)
	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```
At first, I think a physical node has many ports are occupied by a large number of processes.
But I found the total number of ports in use is only 671.
```
[yarnr1115 ~]$ netstat -a | wc -l 671
671
```
I  checked the code of PythonWorkerFactory in line 204 and found:
```
 daemon = pb.start()
 val in = new DataInputStream(daemon.getInputStream)
 try {
 daemonPort = in.readInt()
 } catch {
 case _: EOFException =>
 throw new SparkException(s"No port number in $daemonModule's stdout")
 }
```
I added some code here:
```
logError("Meet EOFException, daemon is alive: ${daemon.isAlive()}")
logError("Exit value: ${daemon.exitValue()}")
```
Then I recurrent the exception and it's message as below:
```
19/10/28 16:15:03 ERROR PythonWorkerFactory: Meet EOFException, daemon is alive: false
19/10/28 16:15:03 ERROR PythonWorkerFactory: Exit value: 139
19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
org.apache.spark.SparkException: No port number in pyspark.daemon's stdout
 at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:206)
 at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122)
 at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95)
 at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117)
 at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108)
 at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
 at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337)
 at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335)
 at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182)
 at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156)
 at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091)
 at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156)
 at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882)
 at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:286)
 at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
 at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
 at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
 at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
 at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
 at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
 at org.apache.spark.scheduler.Task.run(Task.scala:121)
 at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
 at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
 at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
 at java.lang.Thread.run(Thread.java:745)
```
I think the exception message has caused me a lot of confusion.
This PR will add meaningful log for exception information.

### Why are the changes needed?
In order to clarify the exception and try three times default.

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

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

Closes #26510 from beliefer/improve-except-message.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-21 16:13:42 +09:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

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

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06:00
jiake a8d98833b8 [SPARK-29893] improve the local shuffle reader performance by changing the reading task number from 1 to multi
### What changes were proposed in this pull request?
This PR update the local reader task number from 1 to multi `partitionStartIndices.length`.

### Why are the changes needed?
Improve the performance of local shuffle reader.

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

### How was this patch tested?
Existing UTs

Closes #26516 from JkSelf/improveLocalShuffleReader.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-19 19:18:08 +08:00
yudovin 2e71a6e7ba [SPARK-27558][CORE] Gracefully cleanup task when it fails with OOM exception
### What changes were proposed in this pull request?

When a task fails with OOM exception, the `UnsafeInMemorySorter.array` could be `null`. In the meanwhile, the `cleanupResources()` on task completion would call `UnsafeInMemorySorter.getMemoryUsage` in turn, and that lead to another NPE thrown.

### Why are the changes needed?

Check if `array` is null in `UnsafeInMemorySorter.getMemoryUsage` and it should help to avoid NPE.

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

### How was this patch tested?
It was tested manually.

Closes #26349 from ayudovin/fix-npe-in-listener.

Authored-by: yudovin <artsiom.yudovin@profitero.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-11-18 22:05:34 -08:00
Pavithra Ramachandran 5336473004 [SPARK-29476][WEBUI] add tooltip for Thread
### What changes were proposed in this pull request?
Adding tooltip for Thread Dump - Thread Locks

Before:
![Screenshot from 2019-11-04 17-11-22](https://user-images.githubusercontent.com/51401130/68127349-b963f580-ff3b-11e9-8547-e01907382632.png)

After:
![Screenshot from 2019-11-13 18-12-54](https://user-images.githubusercontent.com/51401130/68768698-08e7a700-0649-11ea-804b-2eb4d5f162b4.png)

### Why are the changes needed?
Thread Dump tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience.

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

### How was this patch tested?
Manual

Closes #26386 from PavithraRamachandran/threadDump_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-16 13:20:05 -06:00
HyukjinKwon 17321782de [SPARK-26923][R][SQL][FOLLOW-UP] Show stderr in the exception whenever possible in RRunner
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23977 I made a mistake related to this line: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112)

Previously,

1. the reader iterator for R worker read some initial data eagerly during RDD materialization. So it read the data before actual execution. For some reasons, in this case, it showed standard error from R worker.

2. After that, when error happens during actual execution, stderr wasn't shown: 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL260)

After my change 3725b1324f (diff-71c2cad03f08cb5f6c70462aa4e28d3aL112), it now ignores 1. case and only does 2. of previous code path, because 1. does not happen anymore as I avoided to such eager execution (which is consistent with PySpark code path).

This PR proposes to do only 1.  before/after execution always because It is pretty much possible R worker was failed during actual execution and it's best to show the stderr from R worker whenever possible.

### Why are the changes needed?

It currently swallows standard error from R worker which makes debugging harder.

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

Yes,

```R
df <- createDataFrame(list(list(n=1)))
collect(dapply(df, function(x) {
  stop("asdkjasdjkbadskjbsdajbk")
  x
}, structType("a double")))
```

**Before:**

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 13.0 failed 1 times, most recent failure: Lost task 0.0 in stage 13.0 (TID 13, 192.168.35.193, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (cranshed)
	at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:130)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:118)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:726)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337)
	at org.apache.spark.
```

**After:**

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.35.193, executor driver): org.apache.spark.SparkException: R unexpectedly exited.
R worker produced errors: Error in computeFunc(inputData) : asdkjasdjkbadskjbsdajbk

	at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:144)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:137)
	at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38)
	at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:128)
	at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:113)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegen
```

### How was this patch tested?

Manually tested and unittest was added.

Closes #26517 from HyukjinKwon/SPARK-26923-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-15 11:13:36 +09:00
turbofei ab981f10a6 [SPARK-29857][WEB UI] Defer render the spark UI dataTables
### What changes were proposed in this pull request?
This PR support defer render the spark UI page.
### Why are the changes needed?
When there are many items, such as tasks and application lists, the renderer of dataTables is heavy, we can enable deferRender to optimize it.
See details in https://datatables.net/examples/ajax/defer_render.html
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Not needed.

Closes #26482 from turboFei/SPARK-29857-defer-render.

Authored-by: turbofei <fwang12@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-14 18:16:45 -06:00
Liang-Chi Hsieh 39596b913b [SPARK-29649][SQL] Stop task set if FileAlreadyExistsException was thrown when writing to output file
### What changes were proposed in this pull request?

We already know task attempts that do not clean up output files in staging directory can cause job failure (SPARK-27194). There was proposals trying to fix it by changing output filename, or deleting existing output files. These proposals are not reliable completely.

The difficulty is, as previous failed task attempt wrote the output file, at next task attempt the output file is still under same staging directory, even the output file name is different.

If the job will go to fail eventually, there is no point to re-run the task until max attempts are reached. For the jobs running a lot of time, re-running the task can waste a lot of time.

This patch proposes to let Spark detect such file already exist exception and stop the task set early.

### Why are the changes needed?

For now, if FileAlreadyExistsException is thrown during data writing job in SQL, the job will continue re-running task attempts until max failure number is reached. It is no point for re-running tasks as task attempts will also fail because they can not write to the existing file too. We should stop the task set early.

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

Yes. If FileAlreadyExistsException is thrown during data writing job in SQL, no more task attempts are re-tried and the task set will be stoped early.

### How was this patch tested?

Unit test.

Closes #26312 from viirya/stop-taskset-if-outputfile-exists.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-13 18:01:38 -08:00
Kent Yao 15a72f3755 [SPARK-29287][CORE] Add LaunchedExecutor message to tell driver which executor is ready for making offers
### What changes were proposed in this pull request?

Add `LaunchedExecuto`r message and send it to the driver when the executor if fully constructed, then the driver can assign the associated executor's totalCores to freeCores for making offers.

### Why are the changes needed?
The executors send RegisterExecutor messages to the driver when onStart.

The driver put the executor data in “the ready to serve map” if it could be, then send RegisteredExecutor back to the executor.  The driver now can make an offer to this executor.

But the executor is not fully constructed yet. When it received RegisteredExecutor, it start to construct itself, initializing block manager, maybe register to the local shuffle server in the way of retrying, then start the heart beating to driver ...

The task allocated here may fail if the executor fails to start or cannot get heart beating to the driver in time.

Sometimes, even worse, when dynamic allocation and blacklisting is enabled and when the runtime executor number down to min executor setting, and those executors receive tasks before fully constructed and if any error happens, the application may be blocked or tear down.

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

NO

### How was this patch tested?

Closes #25964 from yaooqinn/SPARK-29287.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-11-13 16:14:12 -08:00
Nishchal Venkataramana 833a9f12e2 [SPARK-24203][CORE] Make executor's bindAddress configurable
### What changes were proposed in this pull request?
With this change, executor's bindAddress is passed as an input parameter for RPCEnv.create.
A previous PR https://github.com/apache/spark/pull/21261 which addressed the same, was using a Spark Conf property to get the bindAddress which wouldn't have worked for multiple executors.
This PR is to enable anyone overriding CoarseGrainedExecutorBackend with their custom one to be able to invoke CoarseGrainedExecutorBackend.main() along with the option to configure bindAddress.

### Why are the changes needed?
This is required when Kernel-based Virtual Machine (KVM)'s are used inside Linux container where the hostname is not the same as container hostname.

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

### How was this patch tested?
Tested by running jobs with executors on KVMs inside a linux container.

Closes #26331 from nishchalv/SPARK-29670.

Lead-authored-by: Nishchal Venkataramana <nishchal@apple.com>
Co-authored-by: nishchal <nishchal@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-11-13 22:01:48 +00:00
Marcelo Vanzin 56a0b5421e [SPARK-29399][CORE] Remove old ExecutorPlugin interface
SPARK-29397 added new interfaces for creating driver and executor
plugins. These were added in a new, more isolated package that does
not pollute the main o.a.s package.

The old interface is now redundant. Since it's a DeveloperApi and
we're about to have a new major release, let's remove it instead of
carrying more baggage forward.

Closes #26390 from vanzin/SPARK-29399.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-13 09:52:40 +09:00
Ankitraj 45e212e161 [SPARK-29570][WEBUI] Improve tooltip for Executor Tab for Shuffle Write,Blacklisted,Logs,Threaddump columns
### What changes were proposed in this pull request?
All tooltips message will display in centre.

### Why are the changes needed?
Some time tooltips will hide the data of column and tooltips display position will be inconsistent in UI.

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

![Screenshot 2019-10-26 at 3 08 51 AM](https://user-images.githubusercontent.com/8948111/67606124-04dd0d80-f79e-11e9-865a-b7e9bffc9890.png)

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

Closes #26263 from 07ARB/SPARK-29570.

Lead-authored-by: Ankitraj <8948111+07ARB@users.noreply.github.com>
Co-authored-by: 07ARB <ankitrajboudh@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-12 18:49:54 -06:00
lajin 5cb05f4100 [SPARK-29298][CORE] Separate block manager heartbeat endpoint from driver endpoint
### What changes were proposed in this pull request?
Executor's heartbeat will send synchronously to BlockManagerMaster to let it know that the block manager is still alive. In a heavy cluster, it will timeout and cause block manager re-register unexpected.
This improvement will separate a heartbeat endpoint from the driver endpoint. In our production environment, this was really helpful to prevent executors from unstable up and down.

### Why are the changes needed?
`BlockManagerMasterEndpoint` handles many events from executors like `RegisterBlockManager`, `GetLocations`, `RemoveShuffle`, `RemoveExecutor` etc. In a heavy cluster/app, it is always busy. The `BlockManagerHeartbeat` event also was handled in this endpoint. We found it may timeout when it's busy. So we add a new endpoint `BlockManagerMasterHeartbeatEndpoint` to handle heartbeat separately.

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

### How was this patch tested?
Exist UTs

Closes #25971 from LantaoJin/SPARK-29298.

Lead-authored-by: lajin <lajin@ebay.com>
Co-authored-by: Alan Jin <lajin@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-12 16:24:48 +08:00
Xingbo Jiang 0346afa8fc [SPARK-29001][CORE] Print events that take too long time to process
### What changes were proposed in this pull request?
Print events that take too long time to process, to help find out what type of events is slow.
Introduce two extra configs:
* **spark.scheduler.listenerbus.logSlowEvent.enabled** Whether to enable log the events that are slow
* **spark.scheduler.listenerbus.logSlowEvent.threshold** The time threshold of whether an event is considered to be slow.

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

Closes #25702 from jiangxb1987/SPARK-29001.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-12 14:08:13 +08:00
Jungtaek Lim (HeartSaVioR) df08e903b5 [SPARK-29755][CORE] Provide @JsonDeserialize for Option[Long] in LogInfo & AttemptInfoWrapper
### What changes were proposed in this pull request?

This patch adds `JsonDeserialize` annotation for the field which type is `Option[Long]` in LogInfo/AttemptInfoWrapper. It hits https://github.com/FasterXML/jackson-module-scala/wiki/FAQ#deserializing-optionint-and-other-primitive-challenges - other existing json models take care of this, but we missed to add annotation to these classes.

### Why are the changes needed?

Without this change, SHS will throw ClassNotFoundException when rebuilding App UI.

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

No.

### How was this patch tested?

Manually tested.

Closes #26397 from HeartSaVioR/SPARK-29755.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-11 15:49:16 -08:00
Luca Canali 2888009d66 [SPARK-29654][CORE] Add configuration to allow disabling registration of static sources to the metrics system
### What changes were proposed in this pull request?
The Spark metrics system produces many different metrics and not all of them are used at the same time. This proposes to introduce a configuration parameter to allow disabling the registration of metrics in the "static sources" category.

### Why are the changes needed?

This allows to reduce the load and clutter on the sink, in the cases when the metrics in question are not needed. The metrics registerd as "static sources" are under the namespaces CodeGenerator and HiveExternalCatalog and can produce a significant amount of data, as they are registered for the driver and executors.

### Does this PR introduce any user-facing change?
It introduces a new configuration parameter `spark.metrics.register.static.sources.enabled`

### How was this patch tested?
Manually tested.

```
$ cat conf/metrics.properties
*.sink.prometheusServlet.class=org.apache.spark.metrics.sink.PrometheusServlet
*.sink.prometheusServlet.path=/metrics/prometheus
master.sink.prometheusServlet.path=/metrics/master/prometheus
applications.sink.prometheusServlet.path=/metrics/applications/prometheus

$ bin/spark-shell

$ curl -s http://localhost:4040/metrics/prometheus/ | grep Hive
metrics_local_1573330115306_driver_HiveExternalCatalog_fileCacheHits_Count 0
metrics_local_1573330115306_driver_HiveExternalCatalog_filesDiscovered_Count 0
metrics_local_1573330115306_driver_HiveExternalCatalog_hiveClientCalls_Count 0
metrics_local_1573330115306_driver_HiveExternalCatalog_parallelListingJobCount_Count 0
metrics_local_1573330115306_driver_HiveExternalCatalog_partitionsFetched_Count 0

$ bin/spark-shell --conf spark.metrics.static.sources.enabled=false
$ curl -s http://localhost:4040/metrics/prometheus/ | grep Hive
```

Closes #26320 from LucaCanali/addConfigRegisterStaticMetrics.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-09 12:13:13 -08:00
Sean Owen 4d9c36d5ba [SPARK-29795][CORE] Explicitly clear registered metrics on MetricSystem shutdown
### What changes were proposed in this pull request?

Explicitly clear registered metrics when `MetricsSystem` shuts down.

### Why are the changes needed?

See https://issues.apache.org/jira/browse/SPARK-29795 for a complete explanation. The TL;DR is there is some evidence this could leak resources after Spark is shut down, and that may be a minor issue in Spark 3+ for apps or tests that re-start SparkContexts in the same JVM.

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

The possible difference here is that, after Spark is stopped, metrics are no longer available. It's unclear to me whether this is intended behavior anyway.

### How was this patch tested?

See https://issues.apache.org/jira/browse/SPARK-29795 for more context:
- Spark 3 already passes tests without this change
- Spark 2.4 does too, as exists in branch-2.4 now
- Spark 2.4 fails tests if metrics 4.x is used, without this change

The last point is not directly relevant, as Spark 2.4 will not use metrics 4.x. It's evidence that it addresses some potential issue, however.

Closes #26427 from srowen/SPARK-29795.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-09 08:47:05 -06:00
Gabor Somogyi 12598e1b93 [MINOR] FsHistoryProvider import cleanup
### What changes were proposed in this pull request?
As it has been discussed in https://github.com/apache/spark/pull/26397#discussion_r343726691 `FsHistoryProvider` import section has to be cleaned up.

### Why are the changes needed?
Unused imports.

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

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

Closes #26436 from gaborgsomogyi/SPARK-29755.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-09 08:40:56 -06:00
HyukjinKwon 4ec04e5ef3 [SPARK-22340][PYTHON] Add a mode to pin Python thread into JVM's
## What changes were proposed in this pull request?

This PR proposes to add **Single threading model design (pinned thread model)** mode which is an experimental mode to sync threads on PVM and JVM. See https://www.py4j.org/advanced_topics.html#using-single-threading-model-pinned-thread

### Multi threading model

Currently, PySpark uses this model. Threads on PVM and JVM are independent. For instance, in a different Python thread, callbacks are received and relevant Python codes are executed. JVM threads are reused when possible.

Py4J will create a new thread every time a command is received and there is no thread available. See the current model we're using - https://www.py4j.org/advanced_topics.html#the-multi-threading-model

One problem in this model is that we can't sync threads on PVM and JVM out of the box. This leads to some problems in particular at some codes related to threading in JVM side. See:
7056e004ee/core/src/main/scala/org/apache/spark/SparkContext.scala (L334)
Due to reusing JVM threads, seems the job groups in Python threads cannot be set in each thread as described in the JIRA.

### Single threading model design (pinned thread model)

This mode pins and syncs the threads on PVM and JVM to work around the problem above. For instance, in the same Python thread, callbacks are received and relevant Python codes are executed. See https://www.py4j.org/advanced_topics.html#the-single-threading-model

Even though this mode can sync threads on PVM and JVM for other thread related code paths,
 this might cause another problem: seems unable to inherit properties as below (assuming multi-thread mode still creates new threads when existing threads are busy, I suspect this issue already exists when multiple jobs are submitted in multi-thread mode; however, it can be always seen in single threading mode):

```bash
$ PYSPARK_PIN_THREAD=true ./bin/pyspark
```

```python
import threading

spark.sparkContext.setLocalProperty("a", "hi")
def print_prop():
    print(spark.sparkContext.getLocalProperty("a"))

threading.Thread(target=print_prop).start()
```

```
None
```

Unlike Scala side:

```scala
spark.sparkContext.setLocalProperty("a", "hi")
new Thread(new Runnable {
  def run() = println(spark.sparkContext.getLocalProperty("a"))
}).start()
```

```
hi
```

This behaviour potentially could cause weird issues but this PR currently does not target this fix this for now since this mode is experimental.

### How does this PR fix?

Basically there are two types of Py4J servers `GatewayServer` and `ClientServer`.  The former is for multi threading and the latter is for single threading. This PR adds a switch to use the latter.

In Scala side:
The logic to select a server is encapsulated in `Py4JServer` and use `Py4JServer` at `PythonRunner` for Spark summit and `PythonGatewayServer` for Spark shell. Each uses `ClientServer` when `PYSPARK_PIN_THREAD` is `true` and `GatewayServer` otherwise.

In Python side:
Simply do an if-else to switch the server to talk. It uses `ClientServer` when `PYSPARK_PIN_THREAD` is `true` and `GatewayServer` otherwise.

This is disabled by default for now.

## How was this patch tested?

Manually tested. This can be tested via:

```python
PYSPARK_PIN_THREAD=true ./bin/pyspark
```

and/or

```bash
cd python
./run-tests --python-executables=python --testnames "pyspark.tests.test_pin_thread"
```

Also, ran the Jenkins tests with `PYSPARK_PIN_THREAD` enabled.

Closes #24898 from HyukjinKwon/pinned-thread.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-08 06:44:58 +09:00
Jungtaek Lim (HeartSaVioR) 252ecd333f [SPARK-29635][SS] Extract base test suites between Kafka micro-batch sink and Kafka continuous sink
### What changes were proposed in this pull request?

This patch leverages V2 continuous memory stream to extract tests from Kafka micro-batch sink suite and continuous sink suite and deduplicate them. These tests are basically doing the same, except how to run and verify the result.

### Why are the changes needed?

We no longer have same tests spotted on two places - brings 300 lines deletion.

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

No.

### How was this patch tested?

Existing UTs.

Closes #26292 from HeartSaVioR/SPARK-29635.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-06 17:08:42 -08:00
Thomas Graves 075cd557f1 [SPARK-29763] Fix Stage UI Page not showing all accumulators in Task Table
### What changes were proposed in this pull request?
Fix the task table UI to show all accumulators.

Below example was creating 2 accumulators
scala> val accum = sc.longAccumulator("My Accumulator")
scala> val accum2 = sc.longAccumulator("My Accumulator")
scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => {
     accum2.add(x)
     accum.add(x)
     })

Before this change, only shows a single on in task table:
![beforefixtaskui](https://user-images.githubusercontent.com/4563792/68225858-b0fcd080-ffb6-11e9-8561-3dc25a81a106.png)

After this change you can see all of them:
![tasktablegood](https://user-images.githubusercontent.com/4563792/68225911-c5d96400-ffb6-11e9-952a-18d3738711d1.png)

### Why are the changes needed?

Its not showing all accumulators now.

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

no

### How was this patch tested?

Manual testing the UI.

Closes #26402 from tgravescs/SPARK-29763.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-05 14:15:14 -08:00
Alessandro Bellina 3cb18d90c4 [SPARK-29151][CORE] Support fractional resources for task resource scheduling
### What changes were proposed in this pull request?
This PR adds the ability for tasks to request fractional resources, in order to be able to execute more than 1 task per resource. For example, if you have 1 GPU in the executor, and the task configuration is 0.5 GPU/task, the executor can schedule two tasks to run on that 1 GPU.

### Why are the changes needed?
Currently there is no good way to share a resource such that multiple tasks can run on a single unit. This allows multiple tasks to share an executor resource.

### Does this PR introduce any user-facing change?
Yes: There is a configuration change where `spark.task.resource.[resource type].amount` can now be fractional.

### How was this patch tested?
Unit tests and manually on standalone mode, and yarn.

Closes #26078 from abellina/SPARK-29151.

Authored-by: Alessandro Bellina <abellina@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-11-05 08:57:43 -06:00
Marcelo Vanzin d51d228048 [SPARK-29397][CORE] Extend plugin interface to include the driver
Spark 2.4 added the ability for executor plugins to be loaded into
Spark (see SPARK-24918). That feature intentionally skipped the
driver to keep changes small, and also because it is possible to
load code into the Spark driver using listeners + configuration.

But that is a bit awkward, because the listener interface does not
provide hooks into a lot of Spark functionality. This change reworks
the executor plugin interface to also extend to the driver.

- there's a "SparkPlugin" main interface that provides APIs to
  load driver and executor components.
- custom metric support (added in SPARK-28091) can be used by
  plugins to register metrics both in the driver process and in
  executors.
- a communication channel now exists that allows the plugin's
  executor components to send messages to the plugin's driver
  component easily, using the existing Spark RPC system.

The latter was a feature intentionally left out of the original
plugin design (also because it didn't include a driver component).

To avoid polluting the "org.apache.spark" namespace, I added the new
interfaces to the "org.apache.spark.api" package, which seems like
a better place in any case. The actual implementation is kept in
an internal package.

The change includes unit tests for the new interface and features,
but I've also been running a custom plugin that extends the new
API in real applications.

Closes #26170 from vanzin/SPARK-29397.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-04 14:33:17 -08:00
shahid 9023c69db8 [SPARK-29590][WEBUI] JDBC/ODBC tab in the spark UI support hide tables, to make it consistent with other tabs
### What changes were proposed in this pull request?

Currently, JDBC/ODBC tab in the WEBUI doesn't support hiding table. Other tabs in the web ui like, Jobs, stages, SQL etc supports hiding table (refer https://github.com/apache/spark/pull/22592).
In this PR, added the support for hide table in the jdbc/odbc tab also.

### Why are the changes needed?
Spark ui about the contents of the form need to have hidden and show features, when the table records very much. Because sometimes you do not care about the record of the table, you just want to see the contents of the next table, but you have to scroll the scroll bar for a long time to see the contents of the next table.

### Does this PR introduce any user-facing change?
No, except support of hide table

### How was this patch tested?
Manually tested
 ![Screenshot 2019-11-01 at 12 10 05 PM](https://user-images.githubusercontent.com/23054875/68007364-61aa5d80-fca1-11e9-841e-c5a7382871fa.png)
![Screenshot 2019-11-01 at 12 10 43 PM](https://user-images.githubusercontent.com/23054875/68007355-5a834f80-fca1-11e9-844a-f4ba1a333db7.png)

Closes #26353 from shahidki31/hideTable.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-04 09:44:10 -06:00
Kent Yao 8cf76f8d61 [SPARK-29285][SHUFFLE] Temporary shuffle files should be able to handle disk failures
### What changes were proposed in this pull request?

The `getFile` method in `DiskBlockManager` may return a file with an existing subdirectory. But when a disk failure occurs on that subdirectory. this file is inaccessible.
Then the FileNotFoundException like the following usually tear down the entire task, which is a bit heavy.
```
java.io.FileNotFoundException: /mnt/dfs/4/yarn/local/usercache/da_haitao/appcache/application_1568691584183_1953115/blockmgr-cc4689f5-eddd-4b99-8af4-4166a86ec30b/10/temp_shuffle_79be5049-d1d5-4a81-8e67-4ef236d3834f (No such file or directory)
	at java.io.FileOutputStream.open0(Native Method)
	at java.io.FileOutputStream.open(FileOutputStream.java:270)
	at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
	at org.apache.spark.storage.DiskBlockObjectWriter.initialize(DiskBlockObjectWriter.scala:103)
	at org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:116)
	at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:249)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.writeSortedFile(ShuffleExternalSorter.java:209)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.closeAndGetSpills(ShuffleExternalSorter.java:416)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.closeAndWriteOutput(UnsafeShuffleWriter.java:230)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:190)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```
This change pre-touch the temporary file to check whether the parent directory is available or not. If NOT, we may try another possibly heathy disk util we reach the max attempts.
### Why are the changes needed?

Re-running the whole task is much heavier than pick another heathy disk to output the temporary results.

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

NO

### How was this patch tested?

ADD UT

Closes #25962 from yaooqinn/SPARK-29285.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-04 18:21:57 +08:00
Sean Owen 19b8c71436 [SPARK-29674][CORE] Update dropwizard metrics to 4.1.x for JDK 9+
### What changes were proposed in this pull request?

Update the version of dropwizard metrics that Spark uses for metrics to 4.1.x, from 3.2.x.

### Why are the changes needed?

This helps JDK 9+ support, per for example https://github.com/dropwizard/metrics/pull/1236

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

No, although downstream users with custom metrics may be affected.

### How was this patch tested?

Existing tests.

Closes #26332 from srowen/SPARK-29674.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 15:13:06 -08:00
Maxim Gekk 80a89873b2 [SPARK-29733][TESTS] Fix wrong order of parameters passed to assertEquals
### What changes were proposed in this pull request?
The `assertEquals` method of JUnit Assert requires the first parameter to be the expected value. In this PR, I propose to change the order of parameters when the expected value is passed as the second parameter.

### Why are the changes needed?
Wrong order of assert parameters confuses when the assert fails and the parameters have special string representation. For example:
```java
assertEquals(input1.add(input2), new CalendarInterval(5, 5, 367200000000L));
```
```
java.lang.AssertionError:
Expected :interval 5 months 5 days 101 hours
Actual   :interval 5 months 5 days 102 hours
```

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

### How was this patch tested?
By existing tests.

Closes #26377 from MaxGekk/fix-order-in-assert-equals.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 11:21:28 -08:00
root1 39fff9258a [SPARK-29452][WEBUI] Improve Storage tab tooltip
### What changes were proposed in this pull request?
Added Tootips for each column in storage tab of Web UI.

### Why are the changes needed?
Tooltips will help users in understanding columns of storage tabs.

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

### How was this patch tested?
Manually Tested.

Closes #26226 from iRakson/storage_tooltip.

Authored-by: root1 <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-01 08:27:34 -05:00
Xingbo Jiang 8207c835b4 Revert "Prepare Spark release v3.0.0-preview-rc2"
This reverts commit 007c873ae3.
2019-10-30 17:45:44 -07:00
Xingbo Jiang 007c873ae3 Prepare Spark release v3.0.0-preview-rc2
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A
2019-10-30 17:42:59 -07:00
Gabor Somogyi 9c817a83fc [SPARK-29637][CORE] Add description to Job SHS web API
### Why are the changes needed?
Starting from Spark 2.3, the SHS REST API endpoint `/applications/<app_id>/jobs/` is not including `description` in the JobData returned. This is not the case until Spark 2.2.

In this PR I've added the mentioned field.

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

Old API response:
```
[ {
  "jobId" : 0,
  "name" : "foreach at <console>:26",
  "submissionTime" : "2019-10-28T12:41:54.301GMT",
  "completionTime" : "2019-10-28T12:41:54.731GMT",
  "stageIds" : [ 0 ],
  "jobGroup" : "test",
  "status" : "SUCCEEDED",
  "numTasks" : 1,
  "numActiveTasks" : 0,
  "numCompletedTasks" : 1,
  "numSkippedTasks" : 0,
  "numFailedTasks" : 0,
  "numKilledTasks" : 0,
  "numCompletedIndices" : 1,
  "numActiveStages" : 0,
  "numCompletedStages" : 1,
  "numSkippedStages" : 0,
  "numFailedStages" : 0,
  "killedTasksSummary" : { }
} ]
```
New API response:
```
[ {
  "jobId" : 0,
  "name" : "foreach at <console>:26",
  "description" : "job",                            <= This is the addition here
  "submissionTime" : "2019-10-28T13:37:24.107GMT",
  "completionTime" : "2019-10-28T13:37:24.613GMT",
  "stageIds" : [ 0 ],
  "jobGroup" : "test",
  "status" : "SUCCEEDED",
  "numTasks" : 1,
  "numActiveTasks" : 0,
  "numCompletedTasks" : 1,
  "numSkippedTasks" : 0,
  "numFailedTasks" : 0,
  "numKilledTasks" : 0,
  "numCompletedIndices" : 1,
  "numActiveStages" : 0,
  "numCompletedStages" : 1,
  "numSkippedStages" : 0,
  "numFailedStages" : 0,
  "killedTasksSummary" : { }
} ]
```

### How was this patch tested?
Extended + existing unit tests.

Manually:
* Open spark-shell
```
scala> sc.setJobGroup("test", "job", false);
scala> val foo = sc.textFile("/user/foo.txt");
foo: org.apache.spark.rdd.RDD[String] = /user/foo.txt MapPartitionsRDD[1] at textFile at <console>:24
scala> foo.foreach(println);
```
* Access REST API `http://SHS-host:port/api/v1/applications/<app-id>/jobs/`

Closes #26295 from gaborgsomogyi/SPARK-29637.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-29 12:00:52 -07:00
Xingbo Jiang b33a58c0c6 Revert "Prepare Spark release v3.0.0-preview-rc1"
This reverts commit 5eddbb5f1d.
2019-10-28 22:32:34 -07:00
Xingbo Jiang 5eddbb5f1d Prepare Spark release v3.0.0-preview-rc1
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

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

No

### How was this patch tested?

N/A

Closes #26243 from jiangxb1987/3.0.0-preview-prepare.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-28 22:31:29 -07:00