aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-20385][WEB-UI] Submitted Time' field, the date format needs to be ↵郭小龙 102076332017-04-232-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | formatted, in running Drivers table or Completed Drivers table in master web ui. ## What changes were proposed in this pull request? Submitted Time' field, the date format **needs to be formatted**, in running Drivers table or Completed Drivers table in master web ui. Before fix this problem e.g. Completed Drivers Submission ID **Submitted Time** Worker State Cores Memory Main Class driver-20170419145755-0005 **Wed Apr 19 14:57:55 CST 2017** worker-20170419145250-zdh120-40412 FAILED 1 1024.0 MB cn.zte.HdfsTest please see the attachment:https://issues.apache.org/jira/secure/attachment/12863977/before_fix.png After fix this problem e.g. Completed Drivers Submission ID **Submitted Time** Worker State Cores Memory Main Class driver-20170419145755-0006 **2017/04/19 16:01:25** worker-20170419145250-zdh120-40412 FAILED 1 1024.0 MB cn.zte.HdfsTest please see the attachment:https://issues.apache.org/jira/secure/attachment/12863976/after_fix.png 'Submitted Time' field, the date format **has been formatted**, in running Applications table or Completed Applicationstable in master web ui, **it is correct.** e.g. Running Applications Application ID Name Cores Memory per Executor **Submitted Time** User State Duration app-20170419160910-0000 (kill) SparkSQL::10.43.183.120 1 5.0 GB **2017/04/19 16:09:10** root RUNNING 53 s **Format after the time easier to observe, and consistent with the applications table,so I think it's worth fixing.** ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn> Author: guoxiaolong <guo.xiaolong1@zte.com.cn> Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn> Closes #17682 from guoxiaolongzte/SPARK-20385.
* [SPARK-20386][SPARK CORE] modify the log info if the block exists on the ↵eatoncys2017-04-221-17/+35
| | | | | | | | | | | | | | | | | | | | slave already ## What changes were proposed in this pull request? Modify the added memory size to memSize-originalMemSize if the block exists on the slave already since if the block exists, the added memory size should be memSize-originalMemSize; if originalMemSize is bigger than memSize ,then the log info should be Removed memory, removed size should be originalMemSize-memSize ## How was this patch tested? Multiple runs on existing unit tests (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: eatoncys <chen.yanshan@zte.com.cn> Closes #17683 from eatoncys/SPARK-20386.
* [SPARK-20172][CORE] Add file permission check when listing files in ↵jerryshao2017-04-204-19/+145
| | | | | | | | | | | | | | | | | | FsHistoryProvider ## What changes were proposed in this pull request? In the current Spark's HistoryServer we expected to get `AccessControlException` during listing all the files, but unfortunately it was not worked because we actually doesn't check the access permission and no other calls will throw such exception. What was worse is that this check will be deferred until reading files, which is not necessary and quite verbose, since it will be printed out the exception in every 10 seconds when checking the files. So here with this fix, we actually check the read permission during listing the files, which could avoid unnecessary file read later on and suppress the verbose log. ## How was this patch tested? Add unit test to verify. Author: jerryshao <sshao@hortonworks.com> Closes #17495 from jerryshao/SPARK-20172.
* [SPARK-20358][CORE] Executors failing stage on interrupted exception thrown ↵Eric Liang2017-04-202-10/+19
| | | | | | | | | | | | | | | | | | | | by cancelled tasks ## What changes were proposed in this pull request? This was a regression introduced by my earlier PR here: https://github.com/apache/spark/pull/17531 It turns out NonFatal() does not in fact catch InterruptedException. ## How was this patch tested? Extended cancellation unit test coverage. The first test fails before this patch. cc JoshRosen mridulm Author: Eric Liang <ekl@databricks.com> Closes #17659 from ericl/spark-20358.
* [SPARK-20354][CORE][REST-API] When I request access to the 'http: ↵郭小龙 102076332017-04-181-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | //ip:port/api/v1/applications' link, return 'sparkUser' is empty in REST API. ## What changes were proposed in this pull request? When I request access to the 'http: //ip:port/api/v1/applications' link, get the json. I need the 'sparkUser' field specific value, because my Spark big data management platform needs to filter through this field which user submits the application to facilitate my administration and query, but the current return of the json string is empty, causing me this Function can not be achieved, that is, I do not know who the specific application is submitted by this REST Api. **current return json:** [ { "id" : "app-20170417152053-0000", "name" : "KafkaWordCount", "attempts" : [ { "startTime" : "2017-04-17T07:20:51.395GMT", "endTime" : "1969-12-31T23:59:59.999GMT", "lastUpdated" : "2017-04-17T07:20:51.395GMT", "duration" : 0, **"sparkUser" : "",** "completed" : false, "endTimeEpoch" : -1, "startTimeEpoch" : 1492413651395, "lastUpdatedEpoch" : 1492413651395 } ] } ] **When I fix this question, return json:** [ { "id" : "app-20170417154201-0000", "name" : "KafkaWordCount", "attempts" : [ { "startTime" : "2017-04-17T07:41:57.335GMT", "endTime" : "1969-12-31T23:59:59.999GMT", "lastUpdated" : "2017-04-17T07:41:57.335GMT", "duration" : 0, **"sparkUser" : "mr",** "completed" : false, "startTimeEpoch" : 1492414917335, "endTimeEpoch" : -1, "lastUpdatedEpoch" : 1492414917335 } ] } ] ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn> Author: guoxiaolong <guo.xiaolong1@zte.com.cn> Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn> Closes #17656 from guoxiaolongzte/SPARK-20354.
* [SPARK-20344][SCHEDULER] Duplicate call in ↵Robert Stupp2017-04-181-16/+16
| | | | | | | | | | | | | | | | FairSchedulableBuilder.addTaskSetManager ## What changes were proposed in this pull request? Eliminate the duplicate call to `Pool.getSchedulableByName()` in `FairSchedulableBuilder.addTaskSetManager` ## How was this patch tested? ./dev/run-tests Author: Robert Stupp <snazy@snazy.de> Closes #17647 from snazy/20344-dup-call-master.
* [SPARK-20343][BUILD] Add avro dependency in core POM to resolve build ↵hyukjinkwon2017-04-161-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failure in SBT Hadoop 2.6 master on Jenkins ## What changes were proposed in this pull request? This PR proposes to add ``` <dependency> <groupId>org.apache.avro</groupId> <artifactId>avro</artifactId> </dependency> ``` in core POM to see if it resolves the build failure as below: ``` [error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.6/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala:123: value createDatumWriter is not a member of org.apache.avro.generic.GenericData [error] writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema)) [error] ``` https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/2770/consoleFull ## How was this patch tested? I tried many ways but I was unable to reproduce this in my local. Sean also tried the way I did but he was also unable to reproduce this. Please refer the comments in https://github.com/apache/spark/pull/17477#issuecomment-294094092 Author: hyukjinkwon <gurwls223@gmail.com> Closes #17642 from HyukjinKwon/SPARK-20343.
* [SPARK-20284][CORE] Make {Des,S}erializationStream extend CloseableSergei Lebedev2017-04-131-4/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR allows to use `SerializationStream` and `DeserializationStream` in try-with-resources. ## How was this patch tested? `core` unit tests. Author: Sergei Lebedev <s.lebedev@criteo.com> Closes #17598 from superbobry/compression-stream-closeable.
* [SPARK-20131][CORE] Don't use `this` lock in StandaloneSchedulerBackend.stopShixiong Zhu2017-04-121-16/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `o.a.s.streaming.StreamingContextSuite.SPARK-18560 Receiver data should be deserialized properly` is flaky is because there is a potential dead-lock in StandaloneSchedulerBackend which causes `await` timeout. Here is the related stack trace: ``` "Thread-31" #211 daemon prio=5 os_prio=31 tid=0x00007fedd4808000 nid=0x16403 waiting on condition [0x00007000239b7000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x000000079b49ca10> (a scala.concurrent.impl.Promise$CompletionLatch) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:208) at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:218) at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223) at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:201) at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:92) at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:76) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.stop(CoarseGrainedSchedulerBackend.scala:402) at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.org$apache$spark$scheduler$cluster$StandaloneSchedulerBackend$$stop(StandaloneSchedulerBackend.scala:213) - locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.stop(StandaloneSchedulerBackend.scala:116) - locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.TaskSchedulerImpl.stop(TaskSchedulerImpl.scala:517) at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1657) at org.apache.spark.SparkContext$$anonfun$stop$8.apply$mcV$sp(SparkContext.scala:1921) at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1302) at org.apache.spark.SparkContext.stop(SparkContext.scala:1920) at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:708) at org.apache.spark.streaming.StreamingContextSuite$$anonfun$43$$anonfun$apply$mcV$sp$66$$anon$3.run(StreamingContextSuite.scala:827) "dispatcher-event-loop-3" #18 daemon prio=5 os_prio=31 tid=0x00007fedd603a000 nid=0x6203 waiting for monitor entry [0x0000700003be4000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint.org$apache$spark$scheduler$cluster$CoarseGrainedSchedulerBackend$DriverEndpoint$$makeOffers(CoarseGrainedSchedulerBackend.scala:253) - waiting to lock <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anonfun$receive$1.applyOrElse(CoarseGrainedSchedulerBackend.scala:124) at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117) at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205) at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101) at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:213) 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 PR removes `synchronized` and changes `stopping` to AtomicBoolean to ensure idempotent to fix the dead-lock. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17610 from zsxwing/SPARK-20131.
* [SPARK-15354][FLAKY-TEST] TopologyAwareBlockReplicationPolicyBehavior.Peers ↵Wenchen Fan2017-04-131-2/+11
| | | | | | | | | | | | | | | | | | in 2 racks ## What changes were proposed in this pull request? `TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks` is failing occasionally: https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.storage.TopologyAwareBlockReplicationPolicyBehavior&test_name=Peers+in+2+racks. This is because, when we generate 10 block manager id to test, they may all belong to the same rack, as the rack is randomly picked. This PR fixes this problem by forcing each rack to be picked at least once. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #17624 from cloud-fan/test.
* [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkinshyukjinkwon2017-04-1211-21/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable. There are several problems with it: - It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?". - > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up. (see joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627)) To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above. There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013 Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings. ## How was this patch tested? Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`. This was tested via manually adding `time.time()` as below: ```diff profiles_and_goals = build_profiles + sbt_goals print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ", " ".join(profiles_and_goals)) + import time + st = time.time() exec_sbt(profiles_and_goals) + print("Elapsed :[%s]" % str(time.time() - st)) ``` produces ``` ... ======================================================================== Building Unidoc API Documentation ======================================================================== ... [info] Main Java API documentation successful. ... Elapsed :[94.8746569157] ... Author: hyukjinkwon <gurwls223@gmail.com> Closes #17477 from HyukjinKwon/SPARK-18692.
* [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish ↵Sean Owen2017-04-1022-44/+64
| | | | | | | | | | | | | | | | | | locale bug" causes Spark problems ## What changes were proposed in this pull request? Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17527 from srowen/SPARK-20156.
* [SPARK-20243][TESTS] DebugFilesystem.assertNoOpenStreams thread raceBogdan Raducanu2017-04-101-10/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Synchronize access to openStreams map. ## How was this patch tested? Existing tests. Author: Bogdan Raducanu <bogdan@databricks.com> Closes #17592 from bogdanrdc/SPARK-20243.
* [SPARK-20260][MLLIB] String interpolation required for error messageVijay Ramesh2017-04-091-1/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This error message doesn't get properly formatted because of a missing `s`. Currently the error looks like: ``` Caused by: java.lang.IllegalArgumentException: requirement failed: indices should be one-based and in ascending order; found current=$current, previous=$previous; line="$line" ``` (note the literal `$current` instead of the interpolated value) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Vijay Ramesh <vramesh@demandbase.com> Closes #17572 from vijaykramesh/master.
* [SPARK-19991][CORE][YARN] FileSegmentManagedBuffer performance improvementSean Owen2017-04-091-1/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid `NoSuchElementException` every time `ConfigProvider.get(val, default)` falls back to default. This apparently causes non-trivial overhead in at least one path, and can easily be avoided. See https://github.com/apache/spark/pull/17329 ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #17567 from srowen/SPARK-19991.
* [MINOR] Issue: Change "slice" vs "partition" in exception messages (and code?)asmith262017-04-091-1/+1
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Came across the term "slice" when running some spark scala code. Consequently, a Google search indicated that "slices" and "partitions" refer to the same things; indeed see: - [This issue](https://issues.apache.org/jira/browse/SPARK-1701) - [This pull request](https://github.com/apache/spark/pull/2305) - [This StackOverflow answer](http://stackoverflow.com/questions/23436640/what-is-the-difference-between-an-rdd-partition-and-a-slice) and [this one](http://stackoverflow.com/questions/24269495/what-are-the-differences-between-slices-and-partitions-of-rdds) Thus this pull request fixes the occurrence of slice I came accross. Nonetheless, [it would appear](https://github.com/apache/spark/search?utf8=%E2%9C%93&q=slice&type=) there are still many references to "slice/slices" - thus I thought I'd raise this Pull Request to address the issue (sorry if this is the wrong place, I'm not too familar with raising apache issues). ## How was this patch tested? (Not tested locally - only a minor exception message change.) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: asmith26 <asmith26@users.noreply.github.com> Closes #17565 from asmith26/master.
* [SPARK-17019][CORE] Expose on-heap and off-heap memory usage in various placesjerryshao2017-04-0622-118/+628
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992), Spark supports persisting data into off-heap memory, but the usage of on-heap and off-heap memory is not exposed currently, it is not so convenient for user to monitor and profile, so here propose to expose off-heap memory as well as on-heap memory usage in various places: 1. Spark UI's executor page will display both on-heap and off-heap memory usage. 2. REST request returns both on-heap and off-heap memory. 3. Also this can be gotten from MetricsSystem. 4. Last this usage can be obtained programmatically from SparkListener. Attach the UI changes: ![screen shot 2016-08-12 at 11 20 44 am](https://cloud.githubusercontent.com/assets/850797/17612032/6c2f4480-607f-11e6-82e8-a27fb8cbb4ae.png) Backward compatibility is also considered for event-log and REST API. Old event log can still be replayed with off-heap usage displayed as 0. For REST API, only adds the new fields, so JSON backward compatibility can still be kept. ## How was this patch tested? Unit test added and manual verification. Author: jerryshao <sshao@hortonworks.com> Closes #14617 from jerryshao/SPARK-17019.
* [SPARK-20217][CORE] Executor should not fail stage if killed task throws ↵Eric Liang2017-04-052-2/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | non-interrupted exception ## What changes were proposed in this pull request? If tasks throw non-interrupted exceptions on kill (e.g. java.nio.channels.ClosedByInterruptException), their death is reported back as TaskFailed instead of TaskKilled. This causes stage failure in some cases. This is reproducible as follows. Run the following, and then use SparkContext.killTaskAttempt to kill one of the tasks. The entire stage will fail since we threw a RuntimeException instead of InterruptedException. ``` spark.range(100).repartition(100).foreach { i => try { Thread.sleep(10000000) } catch { case t: InterruptedException => throw new RuntimeException(t) } } ``` Based on the code in TaskSetManager, I think this also affects kills of speculative tasks. However, since the number of speculated tasks is few, and usually you need to fail a task a few times before the stage is cancelled, it unlikely this would be noticed in production unless both speculation was enabled and the num allowed task failures was = 1. We should probably unconditionally return TaskKilled instead of TaskFailed if the task was killed by the driver, regardless of the actual exception thrown. ## How was this patch tested? Unit test. The test fails before the change in Executor.scala cc JoshRosen Author: Eric Liang <ekl@databricks.com> Closes #17531 from ericl/fix-task-interrupt.
* [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default ↵Dilip Biswal2017-04-063-0/+34
| | | | | | | | | | | | | | timezone settings ## What changes were proposed in this pull request? Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf. ## How was this patch tested? Added an unit test in ConfigEntrySuite Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #17537 from dilipbiswal/timezone_debug.
* [SPARK-19807][WEB UI] Add reason for cancellation when a stage is killed ↵shaolinliu2017-04-051-1/+1
| | | | | | | | | | | | | | | | | | | using web UI ## What changes were proposed in this pull request? When a user kills a stage using web UI (in Stages page), StagesTab.handleKillRequest requests SparkContext to cancel the stage without giving a reason. SparkContext has cancelStage(stageId: Int, reason: String) that Spark could use to pass the information for monitoring/debugging purposes. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: shaolinliu <liu.shaolin1@zte.com.cn> Author: lvdongr <lv.dongdong@zte.com.cn> Closes #17258 from shaolinliu/SPARK-19807.
* [SPARK-20042][WEB UI] Fix log page buttons for reverse proxy modeOliver Köth2017-04-051-3/+16
| | | | | | | | | | | | | | | with spark.ui.reverseProxy=true, full path URLs like /log will point to the master web endpoint which is serving the worker UI as reverse proxy. To access a REST endpoint in the worker in reverse proxy mode , the leading /proxy/"target"/ part of the base URI must be retained. Added logic to log-view.js to handle this, similar to executorspage.js Patch was tested manually Author: Oliver Köth <okoeth@de.ibm.com> Closes #17370 from okoethibm/master.
* [SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]'Denis Bolshakov2017-04-032-0/+17
| | | | | | | | | | | | | | | | | | | [SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]' ## What changes were proposed in this pull request? Array[Int] has been registered in KryoSerializer. The following file has been changed core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala ## How was this patch tested? First, the issue was reproduced by new unit test. Then, the issue was fixed to pass the failed test. Author: Denis Bolshakov <denis.bolshakov@onefactor.com> Closes #17482 from dbolshak/SPARK-9002.
* [SPARK-20084][CORE] Remove internal.metrics.updatedBlockStatuses from ↵Ryan Blue2017-03-311-4/+11
| | | | | | | | | | | | | | | | history files. ## What changes were proposed in this pull request? Remove accumulator updates for internal.metrics.updatedBlockStatuses from SparkListenerTaskEnd entries in the history file. These can cause history files to grow to hundreds of GB because the value of the accumulator contains all tracked blocks. ## How was this patch tested? Current History UI tests cover use of the history file. Author: Ryan Blue <blue@apache.org> Closes #17412 from rdblue/SPARK-20084-remove-block-accumulator-info.
* [SPARK-20127][CORE] few warning have been fixed which Intellij IDEA reported ↵Denis Bolshakov2017-03-3013-29/+22
| | | | | | | | | | | | | | Intellij IDEA ## What changes were proposed in this pull request? Few changes related to Intellij IDEA inspection. ## How was this patch tested? Changes were tested by existing unit tests Author: Denis Bolshakov <denis.bolshakov@onefactor.com> Closes #17458 from dbolshak/SPARK-20127.
* [SPARK-20096][SPARK SUBMIT][MINOR] Expose the right queue name not null if ↵Kent Yao2017-03-302-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | set by --conf or configure file ## What changes were proposed in this pull request? while submit apps with -v or --verbose, we can print the right queue name, but if we set a queue name with `spark.yarn.queue` by --conf or in the spark-default.conf, we just got `null` for the queue in Parsed arguments. ``` bin/spark-shell -v --conf spark.yarn.queue=thequeue Using properties file: /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf .... Adding default property: spark.yarn.queue=default Parsed arguments: master yarn deployMode client ... queue null .... verbose true Spark properties used, including those specified through --conf and those from the properties file /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf: spark.yarn.queue -> thequeue .... ``` ## How was this patch tested? ut and local verify Author: Kent Yao <yaooqinn@hotmail.com> Closes #17430 from yaooqinn/SPARK-20096.
* [DOCS] Docs-only improvementsJacek Laskowski2017-03-309-17/+12
| | | | | | | | | | | | | | | | …adoc ## What changes were proposed in this pull request? Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0). ## How was this patch tested? Local build Author: Jacek Laskowski <jacek@japila.pl> Closes #17417 from jaceklaskowski/window-expression-scaladoc.
* [SPARK-15354][CORE] Topology aware block replication strategiesShubham Chopra2017-03-304-32/+222
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Implementations of strategies for resilient block replication for different resource managers that replicate the 3-replica strategy used by HDFS, where the first replica is on an executor, the second replica within the same rack as the executor and a third replica on a different rack. The implementation involves providing two pluggable classes, one running in the driver that provides topology information for every host at cluster start and the second prioritizing a list of peer BlockManagerIds. The prioritization itself can be thought of an optimization problem to find a minimal set of peers that satisfy certain objectives and replicating to these peers first. The objectives can be used to express richer constraints over and above HDFS like 3-replica strategy. ## How was this patch tested? This patch was tested with unit tests for storage, along with new unit tests to verify prioritization behaviour. Author: Shubham Chopra <schopra31@bloomberg.net> Closes #13932 from shubhamchopra/PrioritizerStrategy.
* [SPARK-20148][SQL] Extend the file commit API to allow subscribing to task ↵Eric Liang2017-03-291-0/+7
| | | | | | | | | | | | | | | | | | | | commit messages ## What changes were proposed in this pull request? The internal FileCommitProtocol interface returns all task commit messages in bulk to the implementation when a job finishes. However, it is sometimes useful to access those messages before the job completes, so that the driver gets incremental progress updates before the job finishes. This adds an `onTaskCommit` listener to the internal api. ## How was this patch tested? Unit tests. cc rxin Author: Eric Liang <ekl@databricks.com> Closes #17475 from ericl/file-commit-api-ext.
* [SPARK-20059][YARN] Use the correct classloader for HBaseCredentialProviderjerryshao2017-03-292-2/+12
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we use system classloader to find HBase jars, if it is specified by `--jars`, then it will be failed with ClassNotFound issue. So here changing to use child classloader. Also putting added jars and main jar into classpath of submitted application in yarn cluster mode, otherwise HBase jars specified with `--jars` will never be honored in cluster mode, and fetching tokens in client side will always be failed. ## How was this patch tested? Unit test and local verification. Author: jerryshao <sshao@hortonworks.com> Closes #17388 from jerryshao/SPARK-20059.
* [SPARK-19556][CORE] Do not encrypt block manager data in memory.Marcelo Vanzin2017-03-2915-254/+686
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change modifies the way block data is encrypted to make the more common cases faster, while penalizing an edge case. As a side effect of the change, all data that goes through the block manager is now encrypted only when needed, including the previous path (broadcast variables) where that did not happen. The way the change works is by not encrypting data that is stored in memory; so if a serialized block is in memory, it will only be encrypted once it is evicted to disk. The penalty comes when transferring that encrypted data from disk. If the data ends up in memory again, it is as efficient as before; but if the evicted block needs to be transferred directly to a remote executor, then there's now a performance penalty, since the code now uses a custom FileRegion implementation to decrypt the data before transferring. This also means that block data transferred between executors now is not encrypted (and thus relies on the network library encryption support for secrecy). Shuffle blocks are still transferred in encrypted form, since they're handled in a slightly different way by the code. This also keeps compatibility with existing external shuffle services, which transfer encrypted shuffle blocks, and avoids having to make the external service aware of encryption at all. The serialization and deserialization APIs in the SerializerManager now do not do encryption automatically; callers need to explicitly wrap their streams with an appropriate crypto stream before using those. As a result of these changes, some of the workarounds added in SPARK-19520 are removed here. Testing: a new trait ("EncryptionFunSuite") was added that provides an easy way to run a test twice, with encryption on and off; broadcast, block manager and caching tests were modified to use this new trait so that the existing tests exercise both encrypted and non-encrypted paths. I also ran some applications with encryption turned on to verify that they still work, including streaming tests that failed without the fix for SPARK-19520. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #17295 from vanzin/SPARK-19556.
* [SPARK-19868] conflict TasksetManager lead to spark stoppedliujianhui2017-03-282-8/+34
| | | | | | | | | | ## What changes were proposed in this pull request? We must set the taskset to zombie before the DAGScheduler handles the taskEnded event. It's possible the taskEnded event will cause the DAGScheduler to launch a new stage attempt (this happens when map output data was lost), and if this happens before the taskSet has been set to zombie, it will appear that we have conflicting task sets. Author: liujianhui <liujianhui@didichuxing> Closes #17208 from liujianhuiouc/spark-19868.
* [SPARK-19803][CORE][TEST] Proactive replication test failuresShubham Chopra2017-03-283-12/+29
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Executors cache a list of their peers that is refreshed by default every minute. The cached stale references were randomly being used for replication. Since those executors were removed from the master, they did not occur in the block locations as reported by the master. This was fixed by 1. Refreshing peer cache in the block manager before trying to pro-actively replicate. This way the probability of replicating to a failed executor is eliminated. 2. Explicitly stopping the block manager in the tests. This shuts down the RPC endpoint use by the block manager. This way, even if a block manager tries to replicate using a stale reference, the replication logic should take care of refreshing the list of peers after failure. ## How was this patch tested? Tested manually Author: Shubham Chopra <schopra31@bloomberg.net> Author: Kay Ousterhout <kayousterhout@gmail.com> Author: Shubham Chopra <shubhamchopra@users.noreply.github.com> Closes #17325 from shubhamchopra/SPARK-19803.
* [SPARK-20088] Do not create new SparkContext in SparkR createSparkContextHossein2017-03-271-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Instead of creating new `JavaSparkContext` we use `SparkContext.getOrCreate`. ## How was this patch tested? Existing tests Author: Hossein <hossein@databricks.com> Closes #17423 from falaki/SPARK-20088.
* logging improvementsJuan Rodriguez Hortala2017-03-262-2/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adding additional information to existing logging messages: - YarnAllocator: log the executor ID together with the container id when a container for an executor is launched. - NettyRpcEnv: log the receiver address when there is a timeout waiting for an answer to a remote call. - ExecutorAllocationManager: fix a typo in the logging message for the list of executors to be removed. ## How was this patch tested? Build spark and submit the word count example to a YARN cluster using cluster mode Author: Juan Rodriguez Hortala <hortala@amazon.com> Closes #17411 from juanrh/logging-improvements.
* [SPARK-20070][SQL] Fix 2.10 buildHerman van Hovell2017-03-252-3/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? Commit https://github.com/apache/spark/commit/91fa80fe8a2480d64c430bd10f97b3d44c007bcc broke the build for scala 2.10. The commit uses `Regex.regex` field which is not available in Scala 2.10. This PR fixes this. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17420 from hvanhovell/SPARK-20070-2.0.
* [SPARK-20070][SQL] Redact DataSourceScanExec treeStringHerman van Hovell2017-03-244-8/+53
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The explain output of `DataSourceScanExec` can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non privileged users. This PR addresses this by adding a redaction facility for the `DataSourceScanExec.treeString`. A user can enable this by setting a regex in the `spark.redaction.string.regex` configuration. ## How was this patch tested? Added a unit test to check the output of DataSourceScanExec. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17397 from hvanhovell/SPARK-20070.
* [SPARK-19820][CORE] Add interface to kill tasks w/ a reasonEric Liang2017-03-2337-108/+269
| | | | | | | | | | | | | | | | | | | | | | | | This commit adds a killTaskAttempt method to SparkContext, to allow users to kill tasks so that they can be re-scheduled elsewhere. This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI. Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`. cc rxin In the stage overview UI the reasons are summarized: ![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png) Within the stage UI you can see individual task kill reasons: ![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png) Existing tests, tried killing some stages in the UI and verified the messages are as expected. Author: Eric Liang <ekl@databricks.com> Author: Eric Liang <ekl@google.com> Closes #17166 from ericl/kill-reason.
* [SPARK-16929] Improve performance when check speculatable tasks.jinxing2017-03-235-6/+176
| | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. Use a MedianHeap to record durations of successful tasks. When check speculatable tasks, we can get the median duration with O(1) time complexity. 2. `checkSpeculatableTasks` will synchronize `TaskSchedulerImpl`. If `checkSpeculatableTasks` doesn't finish with 100ms, then the possibility exists for that thread to release and then immediately re-acquire the lock. Change `scheduleAtFixedRate` to be `scheduleWithFixedDelay` when call method of `checkSpeculatableTasks`. ## How was this patch tested? Added MedianHeapSuite. Author: jinxing <jinxing6042@126.com> Closes #16867 from jinxing64/SPARK-16929.
* [SPARK-19567][CORE][SCHEDULER] Support some Schedulable variables ↵erenavsarogullari2017-03-237-39/+58
| | | | | | | | | | | | | | | | | | | | immutability and access ## What changes were proposed in this pull request? Some `Schedulable` Entities(`Pool` and `TaskSetManager`) variables need refactoring for _immutability_ and _access modifiers_ levels as follows: - From `var` to `val` (if there is no requirement): This is important to support immutability as much as possible. - Sample => `Pool`: `weight`, `minShare`, `priority`, `name` and `taskSetSchedulingAlgorithm`. - Access modifiers: Specially, `var`s access needs to be restricted from other parts of codebase to prevent potential side effects. - `TaskSetManager`: `tasksSuccessful`, `totalResultSize`, `calculatedTasks` etc... This PR is related with #15604 and has been created seperatedly to keep patch content as isolated and to help the reviewers. ## How was this patch tested? Added new UTs and existing UT coverage. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #16905 from erenavsarogullari/SPARK-19567.
* [MINOR][BUILD] Fix javadoc8 breakhyukjinkwon2017-03-231-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Several javadoc8 breaks have been introduced. This PR proposes fix those instances so that we can build Scala/Java API docs. ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:6: error: reference not found [error] * <code>flatMapGroupsWithState</code> operations on {link KeyValueGroupedDataset}. [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:10: error: reference not found [error] * Both, <code>mapGroupsWithState</code> and <code>flatMapGroupsWithState</code> in {link KeyValueGroupedDataset} [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:51: error: reference not found [error] * {link GroupStateTimeout.ProcessingTimeTimeout}) or event time (i.e. [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:52: error: reference not found [error] * {link GroupStateTimeout.EventTimeTimeout}). [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:158: error: reference not found [error] * Spark SQL types (see {link Encoder} for more details). [error] ^ [error] .../spark/mllib/target/java/org/apache/spark/ml/fpm/FPGrowthParams.java:26: error: bad use of '>' [error] * Number of partitions (>=1) used by parallel FP-growth. By default the param is not set, and [error] ^ [error] .../spark/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java:30: error: reference not found [error] * {link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState( [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:211: error: reference not found [error] * See {link GroupState} for more details. [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:232: error: reference not found [error] * See {link GroupState} for more details. [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:254: error: reference not found [error] * See {link GroupState} for more details. [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:277: error: reference not found [error] * See {link GroupState} for more details. [error] ^ [error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found [error] * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe. [error] ^ [error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found [error] * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe. [error] ^ [info] 13 errors ``` ``` jekyll 3.3.1 | Error: Unidoc generation failed ``` ## How was this patch tested? Manually via `jekyll build` Author: hyukjinkwon <gurwls223@gmail.com> Closes #17389 from HyukjinKwon/minor-javadoc8-fix.
* [SPARK-19925][SPARKR] Fix SparkR spark.getSparkFiles fails when it was ↵Yanbo Liang2017-03-211-0/+2
| | | | | | | | | | | | | | called on executors. ## What changes were proposed in this pull request? SparkR ```spark.getSparkFiles``` fails when it was called on executors, see details at [SPARK-19925](https://issues.apache.org/jira/browse/SPARK-19925). ## How was this patch tested? Add unit tests, and verify this fix at standalone and yarn cluster. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17274 from yanboliang/spark-19925.
* [SPARK-19998][BLOCK MANAGER] Change the exception log to add RDD id of the ↵jianran.tfh2017-03-211-1/+1
| | | | | | | | | | | | | | | | | related the block ## What changes were proposed in this pull request? "java.lang.Exception: Could not compute split, block $blockId not found" doesn't have the rdd id info, the "BlockManager: Removing RDD $id" has only the RDD id, so it couldn't find that the Exception's reason is the Removing; so it's better block not found Exception add RDD id info ## How was this patch tested? Existing tests Author: jianran.tfh <jianran.tfh@taobao.com> Author: jianran <tanfanhua1984@163.com> Closes #17334 from jianran/SPARK-19998.
* [SPARK-17204][CORE] Fix replicated off heap storageMichael Allman2017-03-215-25/+105
| | | | | | | | | | | | | | | | (Jira: https://issues.apache.org/jira/browse/SPARK-17204) ## What changes were proposed in this pull request? There are a couple of bugs in the `BlockManager` with respect to support for replicated off-heap storage. First, the locally-stored off-heap byte buffer is disposed of when it is replicated. It should not be. Second, the replica byte buffers are stored as heap byte buffers instead of direct byte buffers even when the storage level memory mode is off-heap. This PR addresses both of these problems. ## How was this patch tested? `BlockManagerReplicationSuite` was enhanced to fill in the coverage gaps. It now fails if either of the bugs in this PR exist. Author: Michael Allman <michael@videoamp.com> Closes #16499 from mallman/spark-17204-replicated_off_heap_storage.
* [SPARK-18817][SPARKR][SQL] change derby log output to temp dirFelix Cheung2017-03-191-0/+9
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Passes R `tempdir()` (this is the R session temp dir, shared with other temp files/dirs) to JVM, set System.Property for derby home dir to move derby.log ## How was this patch tested? Manually, unit tests With this, these are relocated to under /tmp ``` # ls /tmp/RtmpG2M0cB/ derby.log ``` And they are removed automatically when the R session is ended. Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16330 from felixcheung/rderby.
* [SPARK-16599][CORE] java.util.NoSuchElementException: None.get at at ↵Sean Owen2017-03-181-2/+2
| | | | | | | | | | | | | | | | | org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask ## What changes were proposed in this pull request? Avoid None.get exception in (rare?) case that no readLocks exist Note that while this would resolve the immediate cause of the exception, it's not clear it is the root problem. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #17290 from srowen/SPARK-16599.
* [SPARK-13369] Add config for number of consecutive fetch failuresSital Kedia2017-03-173-27/+22
| | | | | | | | | | The previously hardcoded max 4 retries per stage is not suitable for all cluster configurations. Since spark retries a stage at the sign of the first fetch failure, you can easily end up with many stage retries to discover all the failures. In particular, two scenarios this value should change are (1) if there are more than 4 executors per node; in that case, it may take 4 retries to discover the problem with each executor on the node and (2) during cluster maintenance on large clusters, where multiple machines are serviced at once, but you also cannot afford total cluster downtime. By making this value configurable, cluster managers can tune this value to something more appropriate to their cluster configuration. Unit tests Author: Sital Kedia <skedia@fb.com> Closes #17307 from sitalkedia/SPARK-13369.
* [SPARK-19946][TESTING] DebugFilesystem.assertNoOpenStreams should report the ↵Bogdan Raducanu2017-03-162-2/+21
| | | | | | | | | | | | | | | open streams to help debugging ## What changes were proposed in this pull request? DebugFilesystem.assertNoOpenStreams throws an exception with a cause exception that actually shows the code line which leaked the stream. ## How was this patch tested? New test in SparkContextSuite to check there is a cause exception. Author: Bogdan Raducanu <bogdan@databricks.com> Closes #17292 from bogdanrdc/SPARK-19946.
* [SPARK-18066][CORE][TESTS] Add Pool usage policies test coverage for FIFO & ↵erenavsarogullari2017-03-152-8/+96
| | | | | | | | | | | | | | | | | | | | | FAIR Schedulers ## What changes were proposed in this pull request? The following FIFO & FAIR Schedulers Pool usage cases need to have unit test coverage : - FIFO Scheduler just uses **root pool** so even if `spark.scheduler.pool` property is set, related pool is not created and `TaskSetManagers` are added to **root pool**. - FAIR Scheduler uses `default pool` when `spark.scheduler.pool` property is not set. This can be happened when - `Properties` object is **null**, - `Properties` object is **empty**(`new Properties()`), - **default pool** is set(`spark.scheduler.pool=default`). - FAIR Scheduler creates a **new pool** with **default values** when `spark.scheduler.pool` property points a **non-existent** pool. This can be happened when **scheduler allocation file** is not set or it does not contain related pool. ## How was this patch tested? New Unit tests are added. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #15604 from erenavsarogullari/SPARK-18066.
* [SPARK-19960][CORE] Move `SparkHadoopWriter` to `internal/io/`jiangxingbo2017-03-155-64/+99
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR introduces the following changes: 1. Move `SparkHadoopWriter` to `core/internal/io/`, so that it's in the same directory with `SparkHadoopMapReduceWriter`; 2. Move `SparkHadoopWriterUtils` to a separated file. After this PR is merged, we may consolidate `SparkHadoopWriter` and `SparkHadoopMapReduceWriter`, and make the new commit protocol support the old `mapred` package's committer; ## How was this patch tested? Tested by existing test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #17304 from jiangxb1987/writer.
* [SPARK-19889][SQL] Make TaskContext callbacks thread safeHerman van Hovell2017-03-153-34/+93
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It is sometimes useful to use multiple threads in a task to parallelize tasks. These threads might register some completion/failure listeners to clean up when the task completes or fails. We currently cannot register such a callback and be sure that it will get called, because the context might be in the process of invoking its callbacks, when the the callback gets registered. This PR improves this by making sure that you cannot add a completion/failure listener from a different thread when the context is being marked as completed/failed in another thread. This is done by synchronizing these methods on the task context itself. Failure listeners were called only once. Completion listeners now follow the same pattern; this lifts the idempotency requirement for completion listeners and makes it easier to implement them. In some cases we can (accidentally) add a completion/failure listener after the fact, these listeners will be called immediately in order make sure we can safely clean-up after a task. As a result of this change we could make the `failure` and `completed` flags non-volatile. The `isCompleted()` method now uses synchronization to ensure that updates are visible across threads. ## How was this patch tested? Adding tests to `TaskContestSuite` to test adding listeners to a completed/failed context. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17244 from hvanhovell/SPARK-19889.