### What changes were proposed in this pull request?
When reading/writing datetime values that before the rebase switch day, from/to Avro/Parquet files, fail by default and ask users to set a config to explicitly do rebase or not.
### Why are the changes needed?
Rebase or not rebase have different behaviors and we should let users decide it explicitly. In most cases, users won't hit this exception as it only affects ancient datetime values.
### Does this PR introduce _any_ user-facing change?
Yes, now users will see an error when reading/writing dates before 1582-10-15 or timestamps before 1900-01-01 from/to Parquet/Avro files, with an error message to ask setting a config.
### How was this patch tested?
updated tests
Closes#28477 from cloud-fan/rebase.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR caught the `NoSuchElementException` and enriched the error message for `AppStatusStore.applicationInfo()` when Spark is starting up and the application information is unavailable.
### Why are the changes needed?
During the initialization of `SparkContext`, it first starts the Web UI and then set up the `LiveListenerBus` thread for dispatching the `SparkListenerApplicationStart` event (which will trigger writing the requested `ApplicationInfo` to `InMemoryStore`). If the Web UI is accessed before this info's being written to `InMemoryStore`, the following `NoSuchElementException` will be thrown.
```
WARN org.eclipse.jetty.server.HttpChannel: /jobs/
java.util.NoSuchElementException
at java.util.Collections$EmptyIterator.next(Collections.java:4191)
at org.apache.spark.util.kvstore.InMemoryStore$InMemoryIterator.next(InMemoryStore.java:467)
at org.apache.spark.status.AppStatusStore.applicationInfo(AppStatusStore.scala:39)
at org.apache.spark.ui.jobs.AllJobsPage.render(AllJobsPage.scala:266)
at org.apache.spark.ui.WebUI.$anonfun$attachPage$1(WebUI.scala:89)
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)
at org.eclipse.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:255)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1345)
at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
at org.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:753)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:220)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
at org.eclipse.jetty.server.Server.handle(Server.java:505)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:370)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:103)
at org.eclipse.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:698)
at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:804)
at java.lang.Thread.run(Thread.java:748)
```
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Manually tested.
This can be reproduced:
1. `./bin/spark-shell`
2. at the same time, open `http://localhost:4040/jobs/` in your browser with quickly refreshing.
Closes#28444 from xccui/SPARK-31632.
Authored-by: Xingcan Cui <xccui@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR changes HistoryServer to set Content-Type.
I noticed that we will get html as plain text when we access to wrong URLs which represent non-existence appId on HistoryServer.
```
<html>
<head>
<meta http-equiv="Content-type" content="text/html; charset=utf-8"/><meta name="viewport" content="width=device-width, initial-scale=1"/><link rel="stylesheet" href="/static/bootstrap.min.css" type="text/css"/><link rel="stylesheet" href="/static/vis-timeline-graph2d.min.css" type="text/css"/><link rel="stylesheet" href="/static/webui.css" type="text/css"/><link rel="stylesheet" href="/static/timeline-view.css" type="text/css"/><script src="/static/sorttable.js"></script><script src="/static/jquery-3.4.1.min.js"></script><script src="/static/vis-timeline-graph2d.min.js"></script><script src="/static/bootstrap.bundle.min.js"></script><script src="/static/initialize-tooltips.js"></script><script src="/static/table.js"></script><script src="/static/timeline-view.js"></script><script src="/static/log-view.js"></script><script src="/static/webui.js"></script><script>setUIRoot('')</script>
<link rel="shortcut icon" href="/static/spark-logo-77x50px-hd.png"></link>
<title>Not Found</title>
</head>
<body>
<div class="container-fluid">
<div class="row">
<div class="col-12">
<h3 style="vertical-align: middle; display: inline-block;">
<a style="text-decoration: none" href="/">
<img src="/static/spark-logo-77x50px-hd.png"/>
<span class="version" style="margin-right: 15px;">3.1.0-SNAPSHOT</span>
</a>
Not Found
</h3>
</div>
</div>
<div class="row">
<div class="col-12">
<div class="row">Application local-1589239 not found.</div>
</div>
</div>
</div>
</body>
</html>
```
The reason is Content-Type not set. I confirmed it with `curl -I http://localhost:18080/history/<wrong-appId>`
```
HTTP/1.1 404 Not Found
Date: Wed, 13 May 2020 06:59:29 GMT
Cache-Control: no-cache, no-store, must-revalidate
X-Frame-Options: SAMEORIGIN
X-XSS-Protection: 1; mode=block
X-Content-Type-Options: nosniff
Content-Length: 1778
Server: Jetty(9.4.18.v20190429)
```
### Why are the changes needed?
This is a bug.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
I added a test case for this issue.
Closes#28519 from sarutak/fix-content-type.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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-3453https://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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
<!--
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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>
### 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>