[spark] branch branch-2.4 updated: [SPARK-32003][CORE][2.4] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new 39d31dc [SPARK-32003][CORE][2.4] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost 39d31dc is described below commit 39d31dcba362c1c34995d15495d6b3753f8355b5 Author: Wing Yew Poon AuthorDate: Tue Aug 4 14:36:59 2020 -0500 [SPARK-32003][CORE][2.4] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost ### What changes were proposed in this pull request? If an executor is lost, the `DAGScheduler` handles the executor loss by removing the executor but does not unregister its outputs if the external shuffle service is used. However, if the node on which the executor runs is lost, the shuffle service may not be able to serve the shuffle files. In such a case, when fetches from the executor's outputs fail in the same stage, the `DAGScheduler` again removes the executor and by right, should unregister its outputs. It doesn't because the epoch used to track the executor failure has not increased. We track the epoch for failed executors that result in lost file output separately, so we can unregister the outputs in this scenario. The idea to track a second epoch is due to Attila Zsolt Piros. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. This test fails without the change and passes with it. Closes #29182 from wypoon/SPARK-32003-2.4. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../org/apache/spark/scheduler/DAGScheduler.scala | 100 +--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 101 + 2 files changed, 151 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d0d12d8..18baa0b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -171,13 +171,34 @@ private[spark] class DAGScheduler( */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with - // every task. When we detect a node failing, we note the current epoch number and failed - // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. - // - // TODO: Garbage collect information about failure epochs when we know there are no more - // stray messages to detect. - private val failedEpoch = new HashMap[String, Long] + /** + * Tracks the latest epoch of a fully processed error related to the given executor. (We use + * the MapOutputTracker's epoch number, which is sent with every task.) + * + * When an executor fails, it can affect the results of many tasks, and we have to deal with + * all of them consistently. We don't simply ignore all future results from that executor, + * as the failures may have been transient; but we also don't want to "overreact" to follow- + * on errors we receive. Furthermore, we might receive notification of a task success, after + * we find out the executor has actually failed; we'll assume those successes are, in fact, + * simply delayed notifications and the results have been lost, if the tasks started in the + * same or an earlier epoch. In particular, we use this to control when we tell the + * BlockManagerMaster that the BlockManager has been lost. + */ + private val executorFailureEpoch = new HashMap[String, Long] + + /** + * Tracks the latest epoch of a fully processed error where shuffle files have been lost from + * the given executor. + * + * This is closely related to executorFailureEpoch. They only differ for the executor when + * there is an external shuffle service serving shuffle files and we haven't been notified that + * the entire worker has been lost. In that case, when an executor is lost, we do not update + * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only the executor + * fails, we do not unregister the shuffle data as it can still be served; but if there is + * a failure in the shuffle service (resulting in fetch failure), we unregister the shuffle + * data only once, even if we get many fetch failures. + */ + private val shuffleFileLostEpoch = new HashMap[String, Long] private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator @@ -1389,7 +1410,8 @@ private[spark] class DAGScheduler(
[spark] branch branch-3.0 updated: [SPARK-32003][CORE][3.0] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new fd445cb [SPARK-32003][CORE][3.0] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost fd445cb is described below commit fd445cb46ce940fbc68f85d2db14367d9f58739a Author: Wing Yew Poon AuthorDate: Tue Aug 4 11:33:56 2020 -0500 [SPARK-32003][CORE][3.0] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost ### What changes were proposed in this pull request? If an executor is lost, the `DAGScheduler` handles the executor loss by removing the executor but does not unregister its outputs if the external shuffle service is used. However, if the node on which the executor runs is lost, the shuffle service may not be able to serve the shuffle files. In such a case, when fetches from the executor's outputs fail in the same stage, the `DAGScheduler` again removes the executor and by right, should unregister its outputs. It doesn't because the epoch used to track the executor failure has not increased. We track the epoch for failed executors that result in lost file output separately, so we can unregister the outputs in this scenario. The idea to track a second epoch is due to Attila Zsolt Piros. ### Why are the changes needed? Without the changes, the loss of a node could require two stage attempts to recover instead of one. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. This test fails without the change and passes with it. Closes #29193 from wypoon/SPARK-32003-3.0. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../org/apache/spark/scheduler/DAGScheduler.scala | 100 +--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 101 + 2 files changed, 151 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ec6eff3..51445bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -168,13 +168,34 @@ private[spark] class DAGScheduler( */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with - // every task. When we detect a node failing, we note the current epoch number and failed - // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. - // - // TODO: Garbage collect information about failure epochs when we know there are no more - // stray messages to detect. - private val failedEpoch = new HashMap[String, Long] + /** + * Tracks the latest epoch of a fully processed error related to the given executor. (We use + * the MapOutputTracker's epoch number, which is sent with every task.) + * + * When an executor fails, it can affect the results of many tasks, and we have to deal with + * all of them consistently. We don't simply ignore all future results from that executor, + * as the failures may have been transient; but we also don't want to "overreact" to follow- + * on errors we receive. Furthermore, we might receive notification of a task success, after + * we find out the executor has actually failed; we'll assume those successes are, in fact, + * simply delayed notifications and the results have been lost, if the tasks started in the + * same or an earlier epoch. In particular, we use this to control when we tell the + * BlockManagerMaster that the BlockManager has been lost. + */ + private val executorFailureEpoch = new HashMap[String, Long] + + /** + * Tracks the latest epoch of a fully processed error where shuffle files have been lost from + * the given executor. + * + * This is closely related to executorFailureEpoch. They only differ for the executor when + * there is an external shuffle service serving shuffle files and we haven't been notified that + * the entire worker has been lost. In that case, when an executor is lost, we do not update + * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only the executor + * fails, we do not unregister the shuffle data as it can still be served; but if there is + * a failure in the shuffle service (resulting in fetch failure), we unregister the shuffle + * data only once, even if we get many fetch failures. + */ + private val shuffleFileLostEpoch = new HashMap[String, Long]
[spark] branch master updated: [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e8c06af [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost e8c06af is described below commit e8c06af7d1ab19b6478ac542cf876259cac5e13a Author: Wing Yew Poon AuthorDate: Wed Jul 22 09:53:16 2020 -0500 [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost ### What changes were proposed in this pull request? If an executor is lost, the `DAGScheduler` handles the executor loss by removing the executor but does not unregister its outputs if the external shuffle service is used. However, if the node on which the executor runs is lost, the shuffle service may not be able to serve the shuffle files. In such a case, when fetches from the executor's outputs fail in the same stage, the `DAGScheduler` again removes the executor and by right, should unregister its outputs. It doesn't because the epoch used to track the executor failure has not increased. We track the epoch for failed executors that result in lost file output separately, so we can unregister the outputs in this scenario. The idea to track a second epoch is due to Attila Zsolt Piros. ### Why are the changes needed? Without the changes, the loss of a node could require two stage attempts to recover instead of one. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. This test fails without the change and passes with it. Closes #28848 from wypoon/SPARK-32003. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../org/apache/spark/scheduler/DAGScheduler.scala | 100 ++--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 98 2 files changed, 148 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index cb024d0..73c95d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -170,13 +170,34 @@ private[spark] class DAGScheduler( */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with - // every task. When we detect a node failing, we note the current epoch number and failed - // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. - // - // TODO: Garbage collect information about failure epochs when we know there are no more - // stray messages to detect. - private val failedEpoch = new HashMap[String, Long] + /** + * Tracks the latest epoch of a fully processed error related to the given executor. (We use + * the MapOutputTracker's epoch number, which is sent with every task.) + * + * When an executor fails, it can affect the results of many tasks, and we have to deal with + * all of them consistently. We don't simply ignore all future results from that executor, + * as the failures may have been transient; but we also don't want to "overreact" to follow- + * on errors we receive. Furthermore, we might receive notification of a task success, after + * we find out the executor has actually failed; we'll assume those successes are, in fact, + * simply delayed notifications and the results have been lost, if the tasks started in the + * same or an earlier epoch. In particular, we use this to control when we tell the + * BlockManagerMaster that the BlockManager has been lost. + */ + private val executorFailureEpoch = new HashMap[String, Long] + + /** + * Tracks the latest epoch of a fully processed error where shuffle files have been lost from + * the given executor. + * + * This is closely related to executorFailureEpoch. They only differ for the executor when + * there is an external shuffle service serving shuffle files and we haven't been notified that + * the entire worker has been lost. In that case, when an executor is lost, we do not update + * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only the executor + * fails, we do not unregister the shuffle data as it can still be served; but if there is + * a failure in the shuffle service (resulting in fetch failure), we unregister the shuffle + * data only once, even if we get many fetch failures. + */ + private val shuffleFileLostEpoch = new HashMap[String, Long] private [sche
[spark] branch master updated: [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 387ce89 [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration 387ce89 is described below commit 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd Author: Wing Yew Poon AuthorDate: Fri Jan 31 14:28:02 2020 -0600 [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration ### What changes were proposed in this pull request? Add a section to the Configuration page to document configurations for executor metrics. At the same time, rename spark.eventLog.logStageExecutorProcessTreeMetrics.enabled to spark.executor.processTreeMetrics.enabled and make it independent of spark.eventLog.logStageExecutorMetrics.enabled. ### Why are the changes needed? Executor metrics are new in Spark 3.0. They lack documentation. Memory metrics as a whole are always collected, but the ones obtained from the process tree have to be optionally enabled. Making this depend on a single configuration makes for more intuitive behavior. Given this, the configuration property is renamed to better reflect its meaning. ### Does this PR introduce any user-facing change? Yes, only in that the configurations are all new to 3.0. ### How was this patch tested? Not necessary. Closes #27329 from wypoon/SPARK-27324. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../spark/executor/ExecutorMetricsSource.scala | 3 +- .../spark/executor/ProcfsMetricsGetter.scala | 8 ++--- .../org/apache/spark/internal/config/package.scala | 17 +++--- .../spark/deploy/history/HistoryServerSuite.scala | 2 +- docs/configuration.md | 37 ++ docs/monitoring.md | 20 ++-- 6 files changed, 65 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala index b052e43..14645f7 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala @@ -32,8 +32,7 @@ import org.apache.spark.metrics.source.Source * spark.executor.metrics.pollingInterval=. * (2) Procfs metrics are gathered all in one-go and only conditionally: * if the /proc filesystem exists - * and spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true - * and spark.eventLog.logStageExecutorMetrics.enabled=true. + * and spark.executor.processTreeMetrics.enabled=true. */ private[spark] class ExecutorMetricsSource extends Source { diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 0d5dcfb4..80ef757 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -58,11 +58,9 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L logWarning("Exception checking for procfs dir", ioe) false } - val shouldLogStageExecutorMetrics = -SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS) - val shouldLogStageExecutorProcessTreeMetrics = -SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS) - procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && shouldLogStageExecutorMetrics + val shouldPollProcessTreeMetrics = +SparkEnv.get.conf.get(config.EXECUTOR_PROCESS_TREE_METRICS_ENABLED) + procDirExists.get && shouldPollProcessTreeMetrics } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 40b05cf..e68368f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -148,11 +148,8 @@ package object config { private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_PROCESS_TREE_METRICS = -ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled") + .doc("Whether to write per-stage peaks of executor metrics (for each executor) " + +"to the event log.") .booleanConf .createWit
[spark] branch master updated: [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 729f43f [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system 729f43f is described below commit 729f43f499f3dd2718c0b28d73f2ca29cc811eac Author: Luca Canali AuthorDate: Mon Dec 9 08:55:30 2019 -0600 [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system ## What changes were proposed in this pull request? This PR proposes to add instrumentation of memory usage via the Spark Dropwizard/Codahale metrics system. Memory usage metrics are available via the Executor metrics, recently implemented as detailed in https://issues.apache.org/jira/browse/SPARK-23206. Additional notes: This takes advantage of the metrics poller introduced in #23767. ## Why are the changes needed? Executor metrics bring have many useful insights on memory usage, in particular on the usage of storage memory and executor memory. This is useful for troubleshooting. Having the information in the metrics systems allows to add those metrics to Spark performance dashboards and study memory usage as a function of time, as in the example graph https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG ## Does this PR introduce any user-facing change? Adds `ExecutorMetrics` source to publish executor metrics via the Dropwizard metrics system. Details of the available metrics in docs/monitoring.md Adds configuration parameter `spark.metrics.executormetrics.source.enabled` ## How was this patch tested? Tested on YARN cluster and with an existing setup for a Spark dashboard based on InfluxDB and Grafana. Closes #24132 from LucaCanali/memoryMetricsSource. Authored-by: Luca Canali Signed-off-by: Imran Rashid --- .../main/scala/org/apache/spark/SparkContext.scala | 16 +- .../scala/org/apache/spark/executor/Executor.scala | 11 +++- .../spark/executor/ExecutorMetricsPoller.scala | 4 +- .../spark/executor/ExecutorMetricsSource.scala | 65 ++ .../org/apache/spark/internal/config/package.scala | 6 ++ .../spark/metrics/source/SourceConfigSuite.scala | 30 +- docs/monitoring.md | 41 ++ 7 files changed, 167 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0694501..96ca12b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -42,7 +42,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.deploy.StandaloneResourceUtils._ -import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -551,9 +551,16 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = new DAGScheduler(this) _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) +val _executorMetricsSource = + if (_conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) { +Some(new ExecutorMetricsSource) + } else { +None + } + // create and start the heartbeater for collecting memory metrics _heartbeater = new Heartbeater( - () => SparkContext.this.reportHeartBeat(), + () => SparkContext.this.reportHeartBeat(_executorMetricsSource), "driver-heartbeater", conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) _heartbeater.start() @@ -622,6 +629,7 @@ class SparkContext(config: SparkConf) extends Logging { _env.metricsSystem.registerSource(_dagScheduler.metricsSource) _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) _env.metricsSystem.registerSource(new JVMCPUSource()) +_executorMetricsSource.foreach(_.register(_env.metricsSystem)) _executorAllocationManager.foreach { e => _env.metricsSystem.registerSource(e.executorAllocationManagerSource) } @@ -2473,8 +2481,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** Reports heartbeat metrics for the driver. */ - private def reportHeartBeat(): Unit = { + private def reportHeartBeat(executorMetricsSource: Option[ExecutorMetricsSource]): Unit = { val currentMetrics = ExecutorMetrics.getCurrentMetric
[spark] branch master updated: [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2f0a38c [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints 2f0a38c is described below commit 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931 Author: Marcelo Vanzin AuthorDate: Thu Oct 17 13:14:32 2019 -0500 [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints The current RPC backend in Spark supports single- and multi-threaded message delivery to endpoints, but they all share the same underlying thread pool. So an RPC endpoint that blocks a dispatcher thread can negatively affect other endpoints. This can be more pronounced with configurations that limit the number of RPC dispatch threads based on configuration and / or running environment. And exposing the RPC layer to other code (for example with something like SPARK-29396) could make it easy to affect normal Spark operation with a badly written RPC handler. This change adds a new RPC endpoint type that tells the RPC env to create dedicated dispatch threads, so that those effects are minimised. Other endpoints will still need CPU to process their messages, but they won't be able to actively block the dispatch thread of these isolated endpoints. As part of the change, I've changed the most important Spark endpoints (the driver, executor and block manager endpoints) to be isolated from others. This means a couple of extra threads are created on the driver and executor for these endpoints. Tested with existing unit tests, which hammer the RPC system extensively, and also by running applications on a cluster (with a prototype of SPARK-29396). Closes #26059 from vanzin/SPARK-29398. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../executor/CoarseGrainedExecutorBackend.scala| 2 +- .../scala/org/apache/spark/rpc/RpcEndpoint.scala | 16 ++ .../org/apache/spark/rpc/netty/Dispatcher.scala| 130 -- .../scala/org/apache/spark/rpc/netty/Inbox.scala | 6 +- .../org/apache/spark/rpc/netty/MessageLoop.scala | 194 + .../cluster/CoarseGrainedSchedulerBackend.scala| 2 +- .../spark/storage/BlockManagerMasterEndpoint.scala | 4 +- .../spark/storage/BlockManagerSlaveEndpoint.scala | 4 +- .../scala/org/apache/spark/rpc/RpcEnvSuite.scala | 35 +++- .../org/apache/spark/rpc/netty/InboxSuite.scala| 23 +-- 10 files changed, 296 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index fbf2dc7..b4bca1e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -51,7 +51,7 @@ private[spark] class CoarseGrainedExecutorBackend( userClassPath: Seq[URL], env: SparkEnv, resourcesFileOpt: Option[String]) - extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { + extends IsolatedRpcEndpoint with ExecutorBackend with Logging { private implicit val formats = DefaultFormats diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala index 97eed54..4728759 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala @@ -146,3 +146,19 @@ private[spark] trait RpcEndpoint { * [[ThreadSafeRpcEndpoint]] for different messages. */ private[spark] trait ThreadSafeRpcEndpoint extends RpcEndpoint + +/** + * An endpoint that uses a dedicated thread pool for delivering messages. + */ +private[spark] trait IsolatedRpcEndpoint extends RpcEndpoint { + + /** + * How many threads to use for delivering messages. By default, use a single thread. + * + * Note that requesting more than one thread means that the endpoint should be able to handle + * messages arriving from many threads at once, and all the things that entails (including + * messages being delivered to the endpoint out of order). + */ + def threadCount(): Int = 1 + +} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 2f923d7..27c943d 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -17,20 +17,16 @@ package org.apache.spark.rpc.netty -import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} +import
[spark] branch master updated: [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2e28622 [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API 2e28622 is described below commit 2e28622d8aeb9ce2460e803bb7d994196bcc0253 Author: Yifei Huang AuthorDate: Tue Oct 15 12:26:49 2019 -0500 [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API ### What changes were proposed in this pull request? This is the next step of the Spark-25299 work of proposing a new Shuffle storage API. This patch includes the components of the plugin that hook into the driver, including driver shuffle initialization, application cleanup, and shuffle cleanup. ### How was this patch tested? Existing unit tests, plus an additional test for testing the interactions between the driver and executor initialization. Closes #25823 from yifeih/yh/upstream/driver-lifecycle. Lead-authored-by: Yifei Huang Co-authored-by: mccheah Signed-off-by: Imran Rashid --- .../apache/spark/shuffle/api/ShuffleDataIO.java| 6 ++ .../spark/shuffle/api/ShuffleDriverComponents.java | 64 +++ .../shuffle/api/ShuffleExecutorComponents.java | 12 ++- .../shuffle/sort/io/LocalDiskShuffleDataIO.java| 8 +- java => LocalDiskShuffleDriverComponents.java} | 35 +--- .../io/LocalDiskShuffleExecutorComponents.java | 7 +- .../scala/org/apache/spark/ContextCleaner.scala| 8 +- .../main/scala/org/apache/spark/Dependency.scala | 1 + .../main/scala/org/apache/spark/SparkContext.scala | 17 +++- .../apache/spark/shuffle/ShuffleDataIOUtils.scala | 42 ++ .../spark/shuffle/sort/SortShuffleManager.scala| 15 ++-- .../apache/spark/InternalAccumulatorSuite.scala| 3 +- .../shuffle/ShuffleDriverComponentsSuite.scala | 94 ++ 13 files changed, 281 insertions(+), 31 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index e9e50ec..e4554bd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -46,4 +46,10 @@ public interface ShuffleDataIO { * are only invoked on the executors. */ ShuffleExecutorComponents executor(); + + /** + * Called once on driver process to bootstrap the shuffle metadata modules that + * are maintained by the driver. + */ + ShuffleDriverComponents driver(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java new file mode 100644 index 000..b4cec17 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.api; + +import java.util.Map; + +import org.apache.spark.annotation.Private; + +/** + * :: Private :: + * An interface for building shuffle support modules for the Driver. + */ +@Private +public interface ShuffleDriverComponents { + + /** + * Called once in the driver to bootstrap this module that is specific to this application. + * This method is called before submitting executor requests to the cluster manager. + * + * This method should prepare the module with its shuffle components i.e. registering against + * an external file servers or shuffle services, or creating tables in a shuffle + * storage data database. + * + * @return additional SparkConf settings necessary for initializing the executor components. + * This would include configurations that cannot be statically set on the application, like + * the host:port of external services for shuffle storage. + */ + Map initializeApplication(); + + /** + * Called once at the end of the Spark application to clean up any existing shuffle state. + */ +
[spark] branch master updated: [SPARK-27468][CORE] Track correct storage level of RDDs and partitions
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d2f21b0 [SPARK-27468][CORE] Track correct storage level of RDDs and partitions d2f21b0 is described below commit d2f21b019909e66bf49ad764b851b4a65c2438f8 Author: Marcelo Vanzin AuthorDate: Mon Oct 7 16:07:00 2019 -0500 [SPARK-27468][CORE] Track correct storage level of RDDs and partitions Previously, the RDD level would change depending on the status reported by executors for the block they were storing, and individual blocks would reflect that. That is wrong because different blocks may be stored differently in different executors. So now the RDD tracks the user-provided storage level, while the individual partitions reflect the current storage level of that particular block, including the current number of replicas. Closes #25779 from vanzin/SPARK-27468. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../apache/spark/status/AppStatusListener.scala| 20 +++ .../scala/org/apache/spark/status/LiveEntity.scala | 37 +--- .../spark/status/AppStatusListenerSuite.scala | 40 +++--- .../org/apache/spark/status/LiveEntitySuite.scala | 5 +-- 4 files changed, 68 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index c85b3ca..7da0a9d 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -234,8 +234,8 @@ private[spark] class AppStatusListener( (partition.memoryUsed / partition.executors.length) * -1) rdd.diskUsed = addDeltaToValue(rdd.diskUsed, (partition.diskUsed / partition.executors.length) * -1) - partition.update(partition.executors -.filter(!_.equals(event.executorId)), rdd.storageLevel, + partition.update( +partition.executors.filter(!_.equals(event.executorId)), addDeltaToValue(partition.memoryUsed, (partition.memoryUsed / partition.executors.length) * -1), addDeltaToValue(partition.diskUsed, @@ -495,7 +495,7 @@ private[spark] class AppStatusListener( event.stageInfo.rddInfos.foreach { info => if (info.storageLevel.isValid) { -liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info)), now) +liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info, info.storageLevel)), now) } } @@ -916,12 +916,6 @@ private[spark] class AppStatusListener( val diskDelta = event.blockUpdatedInfo.diskSize * (if (storageLevel.useDisk) 1 else -1) val memoryDelta = event.blockUpdatedInfo.memSize * (if (storageLevel.useMemory) 1 else -1) -val updatedStorageLevel = if (storageLevel.isValid) { - Some(storageLevel.description) -} else { - None -} - // We need information about the executor to update some memory accounting values in the // RDD info, so read that beforehand. val maybeExec = liveExecutors.get(executorId) @@ -936,13 +930,9 @@ private[spark] class AppStatusListener( // Update the block entry in the RDD info, keeping track of the deltas above so that we // can update the executor information too. liveRDDs.get(block.rddId).foreach { rdd => - if (updatedStorageLevel.isDefined) { -rdd.setStorageLevel(updatedStorageLevel.get) - } - val partition = rdd.partition(block.name) - val executors = if (updatedStorageLevel.isDefined) { + val executors = if (storageLevel.isValid) { val current = partition.executors if (current.contains(executorId)) { current @@ -957,7 +947,7 @@ private[spark] class AppStatusListener( // Only update the partition if it's still stored in some executor, otherwise get rid of it. if (executors.nonEmpty) { -partition.update(executors, rdd.storageLevel, +partition.update(executors, addDeltaToValue(partition.memoryUsed, memoryDelta), addDeltaToValue(partition.diskUsed, diskDelta)) } else { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index aa4a21c..00c991b 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -30,7 +30,7 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, Ta
[spark] branch master updated: [SPARK-29189][SQL] Add an option to ignore block locations when listing file
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 64fe82b [SPARK-29189][SQL] Add an option to ignore block locations when listing file 64fe82b is described below commit 64fe82b519bdc854fcbef40e906ac1fb181534c2 Author: gwang3 AuthorDate: Mon Oct 7 14:52:55 2019 -0500 [SPARK-29189][SQL] Add an option to ignore block locations when listing file ### What changes were proposed in this pull request? In our PROD env, we have a pure Spark cluster, I think this is also pretty common, where computation is separated from storage layer. In such deploy mode, data locality is never reachable. And there are some configurations in Spark scheduler to reduce waiting time for data locality(e.g. "spark.locality.wait"). While, problem is that, in listing file phase, the location informations of all the files, with all the blocks inside each file, are all fetched from the distributed file system. Actually, in a PROD environment, a table can be so huge that even fetching all these location informations need take tens of seconds. To improve such scenario, Spark need provide an option, where data locality can be totally ignored, all we need in the listing file phase are the files locations, without any block location informations. ### Why are the changes needed? And we made a benchmark in our PROD env, after ignore the block locations, we got a pretty huge improvement. Table Size | Total File Number | Total Block Number | List File Duration(With Block Location) | List File Duration(Without Block Location) -- | -- | -- | -- | -- 22.6T | 3 | 12 | 16.841s | 1.730s 28.8 T | 42001 | 148964 | 10.099s | 2.858s 3.4 T | 2 | 2 | 5.833s | 4.881s ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Via ut. Closes #25869 from wangshisan/SPARK-29189. Authored-by: gwang3 Signed-off-by: Imran Rashid --- .../org/apache/spark/sql/internal/SQLConf.scala| 13 ++ .../execution/datasources/InMemoryFileIndex.scala | 11 ++-- .../sql/execution/datasources/FileIndexSuite.scala | 29 ++ 3 files changed, 51 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9b2e314..3d28b5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -832,6 +832,17 @@ object SQLConf { .intConf .createWithDefault(1) + val IGNORE_DATA_LOCALITY = +buildConf("spark.sql.sources.ignore.datalocality") + .doc("If true, Spark will not fetch the block locations for each file on " + +"listing files. This speeds up file listing, but the scheduler cannot " + +"schedule tasks to take advantage of data locality. It can be particularly " + +"useful if data is read from a remote cluster so the scheduler could never " + +"take advantage of locality anyway.") + .internal() + .booleanConf + .createWithDefault(false) + // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = @@ -2494,6 +2505,8 @@ class SQLConf extends Serializable with Logging { def defaultV2Catalog: Option[String] = getConf(DEFAULT_V2_CATALOG) + def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY) + /** ** SQLConf functionality methods */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index cf7a130..ed860f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -171,6 +171,7 @@ object InMemoryFileIndex extends Logging { areRootPaths: Boolean): Seq[(Path, Seq[FileStatus])] = { val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles +val ignoreLocality = sparkSession.sessionState.conf.ignoreDataLocality // Short-circuits parallel listing when serial listing is likely to be faster. if (paths.size <= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { @@ -181,6 +182,7 @@ object InMemoryFileIndex extends Logging { filter,
[spark] branch master updated: [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 67751e2 [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter 67751e2 is described below commit 67751e26940a16ab6f9950ae66a46b7cb901c102 Author: mcheah AuthorDate: Mon Sep 16 09:08:25 2019 -0500 [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter ### What changes were proposed in this pull request? The previous refactors of the shuffle writers using the shuffle writer plugin resulted in shuffle write metric updates - particularly write times - being lost in particular situations. This patch restores the lost metric updates. ### Why are the changes needed? This fixes a regression. I'm pretty sure that without this, the Spark UI will lose shuffle write time information. ### Does this PR introduce any user-facing change? No change from Spark 2.4. Without this, there would be a user-facing bug in Spark 3.0. ### How was this patch tested? Existing unit tests. Closes #25780 from mccheah/fix-write-metrics. Authored-by: mcheah Signed-off-by: Imran Rashid --- .../apache/spark/shuffle/sort/UnsafeShuffleWriter.java| 2 ++ .../spark/shuffle/ShufflePartitionPairsWriter.scala | 15 --- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index f59bddc..4d11abd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -57,6 +57,7 @@ import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; @@ -382,6 +383,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); OutputStream partitionOutput = writer.openStream(); try { + partitionOutput = new TimeTrackingOutputStream(writeMetrics, partitionOutput); partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); if (compressionCodec != null) { partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala index a988c5e..e0affb8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -21,7 +21,7 @@ import java.io.{Closeable, IOException, OutputStream} import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.api.ShufflePartitionWriter -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.{BlockId, TimeTrackingOutputStream} import org.apache.spark.util.Utils import org.apache.spark.util.collection.PairsWriter @@ -39,6 +39,7 @@ private[spark] class ShufflePartitionPairsWriter( private var isClosed = false private var partitionStream: OutputStream = _ + private var timeTrackingStream: OutputStream = _ private var wrappedStream: OutputStream = _ private var objOut: SerializationStream = _ private var numRecordsWritten = 0 @@ -59,7 +60,8 @@ private[spark] class ShufflePartitionPairsWriter( private def open(): Unit = { try { partitionStream = partitionWriter.openStream - wrappedStream = serializerManager.wrapStream(blockId, partitionStream) + timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, partitionStream) + wrappedStream = serializerManager.wrapStream(blockId, timeTrackingStream) objOut = serializerInstance.serializeStream(wrappedStream) } catch { case e: Exception => @@ -78,6 +80,7 @@ private[spark] class ShufflePartitionPairsWriter( // Setting these to null will prevent the underlying streams from being closed twice // just in case any stream's close() implementation is not idempotent. wrappedStream = null + timeTrackingStream = null partitionStream = null } { // Normally clos
[spark] branch master updated: [SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that sometimes fail
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 151b954 [SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that sometimes fail 151b954 is described below commit 151b954e52c66f78a72530ab69f38100101f6cb7 Author: Wing Yew Poon AuthorDate: Thu Sep 5 15:55:22 2019 -0500 [SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that sometimes fail ### What changes were proposed in this pull request? `ReplayListenerSuite` depends on a listener class to listen for replayed events. This class was implemented by extending `EventLoggingListener`. `EventLoggingListener` does not log executor metrics update events, but uses them to update internal state; on a stage completion event, it then logs stage executor metrics events using this internal state. As executor metrics update events do not get written to the event log, they do not get replayed. The internal state of the replay listene [...] We reimplement the replay listener to simply buffer each and every event it receives. This makes it a simpler yet better tool for verifying the events that get sent through the ReplayListenerBus. ### Why are the changes needed? As explained above. Tests sometimes fail due to events being received by the `EventLoggingListener` that do not get logged (and thus do not get replayed) but influence other events that get logged. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #25673 from wypoon/SPARK-28770. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../spark/scheduler/ReplayListenerSuite.scala | 36 ++ 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index e796137..d65b5cb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -21,12 +21,14 @@ import java.io._ import java.net.URI import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable.ArrayBuffer + import org.apache.hadoop.fs.Path import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfter -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec} import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} @@ -62,7 +64,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) -val eventMonster = new EventMonster(conf) +val eventMonster = new EventBufferingListener try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) @@ -108,7 +110,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) val replayer = new ReplayListenerBus() -val eventMonster = new EventMonster(conf) +val eventMonster = new EventBufferingListener replayer.addListener(eventMonster) // Verify the replay returns the events given the input maybe truncated. @@ -145,7 +147,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) -val eventMonster = new EventMonster(conf) +val eventMonster = new EventBufferingListener try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) @@ -207,7 +209,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // Replay events val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) -val eventMonster = new EventMonster(conf) +val eventMonster = new EventBufferingListener try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) @@ -219,11 +221,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) val originalEvents = sc.eventLogger.get.loggedEvents + .map(JsonProtocol.sparkEventFromJson(_)) val replayedEvents = eventMonster.loggedEvents + .map
[spark-website] branch asf-site updated: CVE-2019-10099
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/spark-website.git The following commit(s) were added to refs/heads/asf-site by this push: new 6e85b26 CVE-2019-10099 6e85b26 is described below commit 6e85b26065809c56a9d317980388032f62a381e8 Author: Imran Rashid AuthorDate: Tue Aug 6 10:14:36 2019 -0500 CVE-2019-10099 ran jekyll build / serve locally. Also updated a random formatting issue on the release process page -- (fixed just by running jekyll build). Author: Imran Rashid Closes #209 from squito/CVE-2019-10099. --- security.md | 24 site/release-process.html | 12 +++- site/security.html| 28 3 files changed, 59 insertions(+), 5 deletions(-) diff --git a/security.md b/security.md index 340622b..7e062b8 100644 --- a/security.md +++ b/security.md @@ -18,6 +18,30 @@ non-public list that will reach the Apache Security team, as well as the Spark P Known Security Issues +CVE-2019-10099: Apache Spark unencrypted data on local disk + +Severity: Important + +Vendor: The Apache Software Foundation + +Versions affected: +- All Spark 1.x, Spark 2.0.x, Spark 2.1.x, and 2.2.x versions +- Spark 2.3.0 to 2.3.2 + +Description: + +Prior to Spark 2.3.3, in certain situations Spark would write user data to local disk unencrypted, even if `spark.io.encryption.enabled=true`. This includes cached blocks that are fetched to disk (controlled by `spark.maxRemoteBlockSizeFetchToMem`); in SparkR, using parallelize; in Pyspark, using broadcast and parallelize; and use of python udfs. + + +Mitigation: + +- 1.x, 2.0.x, 2.1.x, 2.2.x, 2.3.x users should upgrade to 2.3.3 or newer, including 2.4.x + +Credit: + +- This issue was reported by Thomas Graves of NVIDIA. + + CVE-2018-11760: Apache Spark local privilege escalation vulnerability Severity: Important diff --git a/site/release-process.html b/site/release-process.html index 2c5bb41..2e8e82e 100644 --- a/site/release-process.html +++ b/site/release-process.html @@ -299,11 +299,13 @@ changes or in the release news on the website later. Also check that all build and test passes are green from the RISELab Jenkins: https://amplab.cs.berkeley.edu/jenkins/ particularly look for Spark Packaging, QA Compile, QA Test. Note that not all permutations are run on PR therefore it is important to check Jenkins runs. -To cut a release candidate, there are 4 steps: -1. Create a git tag for the release candidate. -1. Package the release binaries sources, and upload them to the Apache staging SVN repo. -1. Create the release docs, and upload them to the Apache staging SVN repo. -1. Publish a snapshot to the Apache staging Maven repo. +To cut a release candidate, there are 4 steps: + + Create a git tag for the release candidate. + Package the release binaries sources, and upload them to the Apache staging SVN repo. + Create the release docs, and upload them to the Apache staging SVN repo. + Publish a snapshot to the Apache staging Maven repo. + The process of cutting a release candidate has been automated via the dev/create-release/do-release-docker.sh script. Run this script, type information it requires, and wait until it finishes. You can also do a single step via the -s option. diff --git a/site/security.html b/site/security.html index 9f5b5b5..4c78b81 100644 --- a/site/security.html +++ b/site/security.html @@ -211,6 +211,34 @@ non-public list that will reach the Apache Security team, as well as the Spark P Known Security Issues +CVE-2019-10099: Apache Spark unencrypted data on local disk + +Severity: Important + +Vendor: The Apache Software Foundation + +Versions affected: + + All Spark 1.x, Spark 2.0.x, Spark 2.1.x, and 2.2.x versions + Spark 2.3.0 to 2.3.2 + + +Description: + +Prior to Spark 2.3.3, in certain situations Spark would write user data to local disk unencrypted, even if spark.io.encryption.enabled=true. This includes cached blocks that are fetched to disk (controlled by spark.maxRemoteBlockSizeFetchToMem); in SparkR, using parallelize; in Pyspark, using broadcast and parallelize; and use of python udfs. + +Mitigation: + + + 1.x, 2.0.x, 2.1.x, 2.2.x, 2.3.x users should upgrade to 2.3.3 or newer, including 2.4.x + + +Credit: + + + This issue was reported by Thomas Graves of NVIDIA. + + CVE-2018-11760: Apache Spark local privilege escalation vulnerability Severity: Important - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to dequeue speculative tasks…
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 70910e6 [SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to dequeue speculative tasks… 70910e6 is described below commit 70910e6ad00f0de4075217d5305d87a477ff1dc4 Author: pgandhi AuthorDate: Tue Jul 30 09:54:51 2019 -0500 [SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to dequeue speculative tasks… … more efficiently This PR improves the performance of scheduling speculative tasks to be O(1) instead of O(numSpeculativeTasks), using the same approach used for scheduling regular tasks. The performance of this method is particularly important because a lock is held on the TaskSchedulerImpl which is a bottleneck for all scheduling operations. We ran a Join query on a large dataset with speculation enabled and out of 10 tasks for the ShuffleMapStage, the maximum number of speculatable tasks that wa [...] In particular, this works by storing a separate stack of tasks by executor, node, and rack locality preferences. Then when trying to schedule a speculative task, rather than scanning all speculative tasks to find ones which match the given executor (or node, or rack) preference, we can jump to a quick check of tasks matching the resource offer. This technique was already used for regular tasks -- this change refactors the code to allow sharing with regular and speculative task execution. ## What changes were proposed in this pull request? Have split the main queue "speculatableTasks" into 5 separate queues based on locality preference similar to how normal tasks are enqueued. Thus, the "dequeueSpeculativeTask" method will avoid performing locality checks for each task at runtime and simply return the preferable task to be executed. ## How was this patch tested? We ran a spark job that performed a join on a 10 TB dataset to test the code change. Original Code: https://user-images.githubusercontent.com/8190/51873321-572df280-2322-11e9-9149-0aae08d5edc6.png;> Optimized Code: https://user-images.githubusercontent.com/8190/51873343-6745d200-2322-11e9-947b-2cfd0f06bcab.png;> As you can see, the run time of the ShuffleMapStage came down from 40 min to 6 min approximately, thus, reducing the overall running time of the spark job by a significant amount. Another example for the same job: Original Code: https://user-images.githubusercontent.com/8190/51873355-70cf3a00-2322-11e9-9c3a-af035449a306.png;> Optimized Code: https://user-images.githubusercontent.com/8190/51873367-7dec2900-2322-11e9-8d07-1b1b49285f71.png;> Closes #23677 from pgandhi999/SPARK-26755. Lead-authored-by: pgandhi Co-authored-by: pgandhi Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSetManager.scala| 292 - .../scheduler/OutputCommitCoordinatorSuite.scala | 12 +- .../spark/scheduler/TaskSetManagerSuite.scala | 122 - 3 files changed, 242 insertions(+), 184 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e7645fc..79a1afc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -131,37 +131,17 @@ private[spark] class TaskSetManager( // same time for a barrier stage. private[scheduler] def isBarrier = taskSet.tasks.nonEmpty && taskSet.tasks(0).isBarrier - // Set of pending tasks for each executor. These collections are actually - // treated as stacks, in which new tasks are added to the end of the - // ArrayBuffer and removed from the end. This makes it faster to detect - // tasks that repeatedly fail because whenever a task failed, it is put - // back at the head of the stack. These collections may contain duplicates - // for two reasons: - // (1): Tasks are only removed lazily; when a task is launched, it remains - // in all the pending lists except the one that it was launched from. - // (2): Tasks may be re-added to these lists multiple times as a result - // of failures. - // Duplicates are handled in dequeueTaskFromList, which ensures that a - // task hasn't already started running before launching it. - private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]] - - // Set of pending tasks for each host. Similar to pendingTasksForExecutor, - // but at host level. - private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // Set of pending tasks for each rack -- similar to the above. - private val pendingTasksForRa
[spark] branch master updated: [SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8313015 [SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver 8313015 is described below commit 8313015e8dd11aafc5b887773e467892d15134de Author: Gabor Somogyi AuthorDate: Wed Jun 26 09:50:54 2019 -0500 [SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver ## What changes were proposed in this pull request? SparkRackResolver generates an INFO message every time is called with 0 arguments. In this PR I've deleted it because it's too verbose. ## How was this patch tested? Existing unit tests + spark-shell. Closes #24935 from gaborgsomogyi/SPARK-28005. Authored-by: Gabor Somogyi Signed-off-by: Imran Rashid --- .../main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala| 3 +++ 1 file changed, 3 insertions(+) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index cab3272..51be932 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -67,6 +67,9 @@ private[spark] class SparkRackResolver(conf: Configuration) extends Logging { } private def coreResolve(hostNames: Seq[String]): Seq[Node] = { +if (hostNames.isEmpty) { + return Seq.empty +} val nodes = new ArrayBuffer[Node] // dnsToSwitchMapping is thread-safe val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-27630][CORE] Properly handle task end events from completed stages
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 38263f6 [SPARK-27630][CORE] Properly handle task end events from completed stages 38263f6 is described below commit 38263f6d153944b6f2f0248d9284861fc82532d6 Author: sychen AuthorDate: Tue Jun 25 14:30:13 2019 -0500 [SPARK-27630][CORE] Properly handle task end events from completed stages ## What changes were proposed in this pull request? Track tasks separately for each stage attempt (instead of tracking by stage), and do NOT reset the numRunningTasks to 0 on StageCompleted. In the case of stage retry, the `taskEnd` event from the zombie stage sometimes makes the number of `totalRunningTasks` negative, which will causes the job to get stuck. Similar problem also exists with `stageIdToTaskIndices` & `stageIdToSpeculativeTaskIndices`. If it is a failed `taskEnd` event of the zombie stage, this will cause `stageIdToTaskIndices` or `stageIdToSpeculativeTaskIndices` to remove the task index of the active stage, and the number of `totalPendingTasks` will increase unexpectedly. ## How was this patch tested? unit test properly handle task end events from completed stages Closes #24497 from cxzl25/fix_stuck_job_follow_up. Authored-by: sychen Signed-off-by: Imran Rashid --- .../apache/spark/ExecutorAllocationManager.scala | 113 - .../org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/SparkListener.scala | 5 +- .../spark/ExecutorAllocationManagerSuite.scala | 33 +- project/MimaExcludes.scala | 6 ++ 5 files changed, 104 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bb95fea..bceb26c 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -491,6 +491,10 @@ private[spark] class ExecutorAllocationManager( numExecutorsToAdd = 1 } + private case class StageAttempt(stageId: Int, stageAttemptId: Int) { +override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)" + } + /** * A listener that notifies the given allocation manager of when to add and remove executors. * @@ -499,29 +503,32 @@ private[spark] class ExecutorAllocationManager( */ private[spark] class ExecutorAllocationListener extends SparkListener { -private val stageIdToNumTasks = new mutable.HashMap[Int, Int] -// Number of running tasks per stage including speculative tasks. +private val stageAttemptToNumTasks = new mutable.HashMap[StageAttempt, Int] +// Number of running tasks per stageAttempt including speculative tasks. // Should be 0 when no stages are active. -private val stageIdToNumRunningTask = new mutable.HashMap[Int, Int] -private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] -// Number of speculative tasks to be scheduled in each stage -private val stageIdToNumSpeculativeTasks = new mutable.HashMap[Int, Int] -// The speculative tasks started in each stage -private val stageIdToSpeculativeTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] - -// stageId to tuple (the number of task with locality preferences, a map where each pair is a -// node and the number of tasks that would like to be scheduled on that node) map, -// maintain the executor placement hints for each stage Id used by resource framework to better -// place the executors. -private val stageIdToExecutorPlacementHints = new mutable.HashMap[Int, (Int, Map[String, Int])] +private val stageAttemptToNumRunningTask = new mutable.HashMap[StageAttempt, Int] +private val stageAttemptToTaskIndices = new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] +// Number of speculative tasks to be scheduled in each stageAttempt +private val stageAttemptToNumSpeculativeTasks = new mutable.HashMap[StageAttempt, Int] +// The speculative tasks started in each stageAttempt +private val stageAttemptToSpeculativeTaskIndices = + new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] + +// stageAttempt to tuple (the number of task with locality preferences, a map where each pair +// is a node and the number of tasks that would like to be scheduled on that node) map, +// maintain the executor placement hints for each stageAttempt used by resource framework +// to better place the executors. +private val stageAttemptToExecutorPlacementHints = + new mutable.HashMap[StageAttempt, (Int, Map[String, Int])]
[spark] branch master updated: [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation.
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b312033 [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation. b312033 is described below commit b312033bd33cd6cbf1f166ccaa7a5df4e3421078 Author: Marcelo Vanzin AuthorDate: Wed Jun 5 08:09:44 2019 -0500 [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation. This change refactors the portions of the ExecutorAllocationManager class that track executor state into a new class, to achieve a few goals: - make the code easier to understand - better separate concerns (task backlog vs. executor state) - less synchronization between event and allocation threads - less coupling between the allocation code and executor state tracking The executor tracking code was moved to a new class (ExecutorMonitor) that encapsulates all the logic of tracking what happens to executors and when they can be timed out. The logic to actually remove the executors remains in the EAM, since it still requires information that is not tracked by the new executor monitor code. In the executor monitor itself, of interest, specifically, is a change in how cached blocks are tracked; instead of polling the block manager, the monitor now uses events to track which executors have cached blocks, and is able to detect also unpersist events and adjust the time when the executor should be removed accordingly. (That's the bug mentioned in the PR title.) Because of the refactoring, a few tests in the old EAM test suite were removed, since they're now covered by the newly added test suite. The EAM suite was also changed a little bit to not instantiate a SparkContext every time. This allowed some cleanup, and the tests also run faster. Tested with new and updated unit tests, and with multiple TPC-DS workloads running with dynamic allocation on; also some manual tests for the caching behavior. Closes #24704 from vanzin/SPARK-20286. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../apache/spark/ExecutorAllocationClient.scala| 9 +- .../apache/spark/ExecutorAllocationManager.scala | 239 +-- .../main/scala/org/apache/spark/SparkContext.scala | 3 +- .../org/apache/spark/internal/config/package.scala | 8 +- .../cluster/CoarseGrainedSchedulerBackend.scala| 4 + .../spark/scheduler/dynalloc/ExecutorMonitor.scala | 283 .../apache/spark/storage/BlockManagerMaster.scala | 9 - .../spark/storage/BlockManagerMasterEndpoint.scala | 31 - .../spark/storage/BlockManagerMessages.scala | 3 - ...g.apache.spark.scheduler.ExternalClusterManager | 1 - .../spark/ExecutorAllocationManagerSuite.scala | 708 ++--- .../deploy/ExternalShuffleServiceDbSuite.scala | 9 +- .../scheduler/dynalloc/ExecutorMonitorSuite.scala | 289 + .../spark/storage/BlockManagerInfoSuite.scala | 14 - .../scheduler/ExecutorAllocationManagerSuite.scala | 2 +- 15 files changed, 822 insertions(+), 790 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index 63d87b4..cb965cb 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -23,11 +23,18 @@ package org.apache.spark */ private[spark] trait ExecutorAllocationClient { - /** Get the list of currently active executors */ private[spark] def getExecutorIds(): Seq[String] /** + * Whether an executor is active. An executor is active when it can be used to execute tasks + * for jobs submitted by the application. + * + * @return whether the executor with the given ID is currently active. + */ + def isExecutorActive(id: String): Boolean + + /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. * @param numExecutors The total number of executors we'd like to have. The cluster manager diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1782027..63df7cc 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ -import org.apache.spark.storage.BlockManagerMaster
[spark] branch master updated: [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d124ce9 [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks d124ce9 is described below commit d124ce9c7ea537c59a776c05977c9f918d38febc Author: Wenchen Fan AuthorDate: Tue May 7 12:02:08 2019 -0500 [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/24375 When `TaskSetManager` skips a task because its corresponding partition is already completed by other `TaskSetManager`s, we should not consider the duration of the task that is finished by other `TaskSetManager`s to schedule the speculative tasks of this `TaskSetManager`. ## How was this patch tested? updated test case Closes #24485 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Imran Rashid --- .../org/apache/spark/scheduler/DAGScheduler.scala | 3 +-- .../apache/spark/scheduler/TaskResultGetter.scala | 5 ++--- .../org/apache/spark/scheduler/TaskScheduler.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 12 --- .../apache/spark/scheduler/TaskSetManager.scala| 24 +++--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 6 ++ .../scheduler/ExternalClusterManagerSuite.scala| 3 +-- .../spark/scheduler/TaskSetManagerSuite.scala | 4 ++-- 8 files changed, 25 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b817eb6..1d4972e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1394,8 +1394,7 @@ private[spark] class DAGScheduler( // finished. Here we notify the task scheduler to skip running tasks for the same partition, // to save resource. if (task.stageAttemptId < stage.latestInfo.attemptNumber()) { - taskScheduler.notifyPartitionCompletion( -stageId, task.partitionId, event.taskInfo.duration) + taskScheduler.notifyPartitionCompletion(stageId, task.partitionId) } task match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 09c4d9b..9b7f901 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -158,10 +158,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul // This method calls `TaskSchedulerImpl.handlePartitionCompleted` asynchronously. We do not want // DAGScheduler to call `TaskSchedulerImpl.handlePartitionCompleted` directly, as it's // synchronized and may hurt the throughput of the scheduler. - def enqueuePartitionCompletionNotification( - stageId: Int, partitionId: Int, taskDuration: Long): Unit = { + def enqueuePartitionCompletionNotification(stageId: Int, partitionId: Int): Unit = { getTaskResultExecutor.execute(() => Utils.logUncaughtExceptions { - scheduler.handlePartitionCompleted(stageId, partitionId, taskDuration) + scheduler.handlePartitionCompleted(stageId, partitionId) }) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1862e16..bfdbf02 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -70,7 +70,7 @@ private[spark] trait TaskScheduler { // Notify the corresponding `TaskSetManager`s of the stage, that a partition has already completed // and they can skip running tasks for it. - def notifyPartitionCompletion(stageId: Int, partitionId: Int, taskDuration: Long) + def notifyPartitionCompletion(stageId: Int, partitionId: Int) // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7e820c3..532eb32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -301,9 +301,8 @@ private[spark] class TaskSchedulerImpl( } } - override def notifyPartitionComp
[spark] branch master updated: [SPARK-13704][CORE][YARN] Reduce rack resolution time
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 52838e7 [SPARK-13704][CORE][YARN] Reduce rack resolution time 52838e7 is described below commit 52838e74afdd58a7c09707284b4e0232dc01ef26 Author: LantaoJin AuthorDate: Mon Apr 8 10:47:06 2019 -0500 [SPARK-13704][CORE][YARN] Reduce rack resolution time ## What changes were proposed in this pull request? When you submit a stage on a large cluster, rack resolving takes a long time when initializing TaskSetManager because a script is invoked to resolve the rack of each host, one by one. Based on current implementation, it takes 30~40 seconds to resolve the racks in our 5000 nodes' cluster. After applied the patch, it decreased to less than 15 seconds. YARN-9332 has added an interface to handle multiple hosts in one invocation to save time. But before upgrading to the newest Hadoop, we could construct the same tool in Spark to resolve this issue. ## How was this patch tested? UT and manually testing on a 5000 node cluster. Closes #24245 from squito/SPARK-13704_update. Lead-authored-by: LantaoJin Co-authored-by: Imran Rashid Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 30 ++-- .../apache/spark/scheduler/TaskSetManager.scala| 33 ++-- .../spark/scheduler/TaskSetManagerSuite.scala | 76 +-- ...calityPreferredContainerPlacementStrategy.scala | 5 +- .../spark/deploy/yarn/SparkRackResolver.scala | 88 -- .../apache/spark/deploy/yarn/YarnAllocator.scala | 2 +- .../apache/spark/deploy/yarn/YarnRMClient.scala| 2 +- .../spark/scheduler/cluster/YarnScheduler.scala| 20 ++--- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 5 +- 9 files changed, 219 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index bffa1ff..e401c39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -158,6 +158,8 @@ private[spark] class TaskSchedulerImpl( private[scheduler] var barrierCoordinator: RpcEndpoint = null + protected val defaultRackValue: Option[String] = None + private def maybeInitBarrierCoordinator(): Unit = { if (barrierCoordinator == null) { barrierCoordinator = new BarrierCoordinator(barrierSyncTimeout, sc.listenerBus, @@ -394,9 +396,10 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(o.executorId) = HashSet[Long]() newExecAvail = true } - for (rack <- getRackForHost(o.host)) { -hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host - } +} +val hosts = offers.map(_.host).toSet.toSeq +for ((host, Some(rack)) <- hosts.zip(getRacksForHosts(hosts))) { + hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do @@ -830,8 +833,25 @@ private[spark] class TaskSchedulerImpl( blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(Set.empty) } - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None + /** + * Get the rack for one host. + * + * Note that [[getRacksForHosts]] should be preferred when possible as that can be much + * more efficient. + */ + def getRackForHost(host: String): Option[String] = { +getRacksForHosts(Seq(host)).head + } + + /** + * Get racks for multiple hosts. + * + * The returned Sequence will be the same length as the hosts argument and can be zipped + * together with the hosts argument. + */ + def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { +hosts.map(_ => defaultRackValue) + } private def waitBackendReady(): Unit = { if (backend.isReady) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 3977c0b..1444220 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -186,8 +186,24 @@ private[spark] class TaskSetManager( // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. - for (i <- (0 until numTasks).reverse) { -addPendingTask(i) + addPendingTasks() + + private def addPendingTasks(): Unit = { +val (_, duration) = Utils.timeTakenMs { + for (i <- (0
[spark] branch master updated: [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 69dd44a [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue 69dd44a is described below commit 69dd44af19ed9a2d7873350e8610c9faeb151a7b Author: LantaoJin AuthorDate: Wed Apr 3 20:09:50 2019 -0500 [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue ## What changes were proposed in this pull request? HighlyCompressedMapStatus uses RoaringBitmap to record the empty blocks. But RoaringBitmap couldn't be ser/deser with unsafe KryoSerializer. It's a bug of RoaringBitmap-0.5.11 and fixed in latest version. This is an update of #24157 ## How was this patch tested? Add a UT Closes #24264 from LantaoJin/SPARK-27216. Lead-authored-by: LantaoJin Co-authored-by: Lantao Jin Signed-off-by: Imran Rashid --- .../spark/serializer/KryoSerializerSuite.scala | 34 ++ dev/deps/spark-deps-hadoop-2.7 | 3 +- dev/deps/spark-deps-hadoop-3.2 | 3 +- pom.xml| 2 +- 4 files changed, 14 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 16eec7e..2442670 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.serializer -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer import java.util.concurrent.Executors @@ -370,30 +370,6 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { assert(thrown.getCause.isInstanceOf[KryoException]) } - test("SPARK-1: deserialize RoaringBitmap throw Buffer underflow exception") { -withTempDir { dir => - val tmpfile = dir.toString + "/RoaringBitmap" - val outStream = new FileOutputStream(tmpfile) - val output = new KryoOutput(outStream) - val bitmap = new RoaringBitmap - bitmap.add(1) - bitmap.add(3) - bitmap.add(5) - // Ignore Kryo because it doesn't use writeObject - bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) - output.flush() - output.close() - - val inStream = new FileInputStream(tmpfile) - val input = new KryoInput(inStream) - val ret = new RoaringBitmap - // Ignore Kryo because it doesn't use readObject - ret.deserialize(new KryoInputObjectInputBridge(null, input)) - input.close() - assert(ret == bitmap) -} - } - test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { val kryo = new KryoSerializer(conf).newKryo() @@ -518,6 +494,14 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { assert(ThreadUtils.awaitResult(f, 10.seconds)) } } + + test("SPARK-27216: test RoaringBitmap ser/dser with Kryo") { +val expected = new RoaringBitmap() +expected.add(1787) +val ser = new KryoSerializer(conf).newInstance() +val actual: RoaringBitmap = ser.deserialize(ser.serialize(expected)) +assert(actual === expected) + } } class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 277274a..de0cb14 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -1,5 +1,5 @@ JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar +RoaringBitmap-0.7.45.jar ST4-4.0.4.jar activation-1.1.1.jar aircompressor-0.10.jar @@ -178,6 +178,7 @@ scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.8.jar scala-xml_2.12-1.0.6.jar shapeless_2.12-2.3.2.jar +shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 5d40c24..326d085 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -1,6 +1,6 @@ HikariCP-java7-2.4.12.jar JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar +RoaringBitmap-0.7.45.jar ST4-4.0.4.jar accessors-smart-1.2.jar activation-1.1.1.jar @@ -198,6 +198,7 @@ scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.8.jar scala-xml_2.12-1.0.6.jar shapeless_2.12-2.3.2.jar +shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar diff --git a/pom.
[spark] branch branch-2.3 updated: [SPARK-27112][CORE] : Create a resource ordering between threads to r…
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.3 by this push: new 7bb2b42 [SPARK-27112][CORE] : Create a resource ordering between threads to r… 7bb2b42 is described below commit 7bb2b42de9a52e75107469f028c1e707d6a174b7 Author: pgandhi AuthorDate: Tue Mar 19 16:22:40 2019 -0500 [SPARK-27112][CORE] : Create a resource ordering between threads to r… …esolve the deadlocks encountered when trying to kill executors either due to dynamic allocation or blacklisting Closes #24072 from pgandhi999/SPARK-27112-2. Authored-by: pgandhi Signed-off-by: Imran Rashid ## What changes were proposed in this pull request? There are two deadlocks as a result of the interplay between three different threads: **task-result-getter thread** **spark-dynamic-executor-allocation thread** **dispatcher-event-loop thread(makeOffers())** The fix ensures ordering synchronization constraint by acquiring lock on `TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in `makeOffers()` as well as killExecutors() method. This ensures resource ordering between the threads and thus, fixes the deadlocks. ## How was this patch tested? Manual Tests Closes #24134 from pgandhi999/branch-2.4-SPARK-27112. Authored-by: pgandhi Signed-off-by: Imran Rashid (cherry picked from commit 95e73b328ac883be2ced9099f20c8878e498e297) Signed-off-by: Imran Rashid --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 13 ++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 8848add..ba45e28 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -238,7 +238,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on all executors private def makeOffers() { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing val activeExecutors = executorDataMap.filterKeys(executorIsAlive) val workOffers = activeExecutors.map { @@ -263,7 +263,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on just one executor private def makeOffers(executorId: String) { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing if (executorIsAlive(executorId)) { val executorData = executorDataMap(executorId) @@ -600,7 +600,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp force: Boolean): Seq[String] = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") -val response = synchronized { +val response = withLock { val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) unknownExecutors.foreach { id => logWarning(s"Executor to kill $id does not exist!") @@ -678,6 +678,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } + + // SPARK-27112: We need to ensure that there is ordering of lock acquisition + // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix + // the deadlock issue exposed in SPARK-27112 + private def withLock[T](fn: => T): T = scheduler.synchronized { +CoarseGrainedSchedulerBackend.this.synchronized { fn } + } } private[spark] object CoarseGrainedSchedulerBackend { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-2.4 updated: [SPARK-27112][CORE] : Create a resource ordering between threads to r…
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new 95e73b3 [SPARK-27112][CORE] : Create a resource ordering between threads to r… 95e73b3 is described below commit 95e73b328ac883be2ced9099f20c8878e498e297 Author: pgandhi AuthorDate: Tue Mar 19 16:22:40 2019 -0500 [SPARK-27112][CORE] : Create a resource ordering between threads to r… …esolve the deadlocks encountered when trying to kill executors either due to dynamic allocation or blacklisting Closes #24072 from pgandhi999/SPARK-27112-2. Authored-by: pgandhi Signed-off-by: Imran Rashid ## What changes were proposed in this pull request? There are two deadlocks as a result of the interplay between three different threads: **task-result-getter thread** **spark-dynamic-executor-allocation thread** **dispatcher-event-loop thread(makeOffers())** The fix ensures ordering synchronization constraint by acquiring lock on `TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in `makeOffers()` as well as killExecutors() method. This ensures resource ordering between the threads and thus, fixes the deadlocks. ## How was this patch tested? Manual Tests Closes #24134 from pgandhi999/branch-2.4-SPARK-27112. Authored-by: pgandhi Signed-off-by: Imran Rashid --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 13 ++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index de7c0d8..aa4e638 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -237,7 +237,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on all executors private def makeOffers() { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing val activeExecutors = executorDataMap.filterKeys(executorIsAlive) val workOffers = activeExecutors.map { @@ -263,7 +263,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on just one executor private def makeOffers(executorId: String) { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing if (executorIsAlive(executorId)) { val executorData = executorDataMap(executorId) @@ -607,7 +607,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp force: Boolean): Seq[String] = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") -val response = synchronized { +val response = withLock { val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) unknownExecutors.foreach { id => logWarning(s"Executor to kill $id does not exist!") @@ -685,6 +685,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } + + // SPARK-27112: We need to ensure that there is ordering of lock acquisition + // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix + // the deadlock issue exposed in SPARK-27112 + private def withLock[T](fn: => T): T = scheduler.synchronized { +CoarseGrainedSchedulerBackend.this.synchronized { fn } + } } private[spark] object CoarseGrainedSchedulerBackend { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-26288][CORE] add initRegisteredExecutorsDB
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8b0aa59 [SPARK-26288][CORE] add initRegisteredExecutorsDB 8b0aa59 is described below commit 8b0aa59218c209d39cbba5959302d8668b885cf6 Author: weixiuli AuthorDate: Tue Mar 19 16:16:43 2019 -0500 [SPARK-26288][CORE] add initRegisteredExecutorsDB ## What changes were proposed in this pull request? As we all know that spark on Yarn uses DB https://github.com/apache/spark/pull/7943 to record RegisteredExecutors information which can be reloaded and used again when the ExternalShuffleService is restarted . The RegisteredExecutors information can't be recorded both in the mode of spark's standalone and spark on k8s , which will cause the RegisteredExecutors information to be lost ,when the ExternalShuffleService is restarted. To solve the problem above, a method is proposed and is committed . ## How was this patch tested? new unit tests Closes #23393 from weixiuli/SPARK-26288. Authored-by: weixiuli Signed-off-by: Imran Rashid --- .../shuffle/ExternalShuffleBlockHandler.java | 5 + core/pom.xml | 7 ++ .../spark/deploy/ExternalShuffleService.scala | 25 +++- .../org/apache/spark/deploy/worker/Worker.scala| 9 ++ .../org/apache/spark/internal/config/package.scala | 7 ++ .../deploy/ExternalShuffleServiceDbSuite.scala | 140 + .../apache/spark/deploy/worker/WorkerSuite.scala | 51 +++- docs/spark-standalone.md | 11 ++ 8 files changed, 253 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index b25e48a..70dcc8b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -66,6 +66,11 @@ public class ExternalShuffleBlockHandler extends RpcHandler { new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } + @VisibleForTesting + public ExternalShuffleBlockResolver getBlockResolver() { +return blockManager; + } + /** Enables mocking out the StreamManager and BlockManager. */ @VisibleForTesting public ExternalShuffleBlockHandler( diff --git a/core/pom.xml b/core/pom.xml index b9f78b2..45bda44 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -372,6 +372,13 @@ tests test + + org.apache.spark + spark-network-shuffle_${scala.binary.version} + ${project.version} + tests + test +
[spark] branch master updated: [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered …
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7043aee [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered … 7043aee is described below commit 7043aee1ba95e92e1cbd0ebafcc5b09b69ee3082 Author: pgandhi AuthorDate: Mon Mar 18 10:33:51 2019 -0500 [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered … …when trying to kill executors either due to dynamic allocation or blacklisting ## What changes were proposed in this pull request? There are two deadlocks as a result of the interplay between three different threads: **task-result-getter thread** **spark-dynamic-executor-allocation thread** **dispatcher-event-loop thread(makeOffers())** The fix ensures ordering synchronization constraint by acquiring lock on `TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in `makeOffers()` as well as killExecutors() method. This ensures resource ordering between the threads and thus, fixes the deadlocks. ## How was this patch tested? Manual Tests Closes #24072 from pgandhi999/SPARK-27112-2. Authored-by: pgandhi Signed-off-by: Imran Rashid --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 13 ++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index dc0f21c..808ef08 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -258,7 +258,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on all executors private def makeOffers() { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing val activeExecutors = executorDataMap.filterKeys(executorIsAlive) val workOffers = activeExecutors.map { @@ -284,7 +284,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on just one executor private def makeOffers(executorId: String) { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing if (executorIsAlive(executorId)) { val executorData = executorDataMap(executorId) @@ -631,7 +631,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp force: Boolean): Seq[String] = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") -val response = synchronized { +val response = withLock { val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) unknownExecutors.foreach { id => logWarning(s"Executor to kill $id does not exist!") @@ -730,6 +730,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp protected def currentDelegationTokens: Array[Byte] = delegationTokens.get() + // SPARK-27112: We need to ensure that there is ordering of lock acquisition + // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix + // the deadlock issue exposed in SPARK-27112 + private def withLock[T](fn: => T): T = scheduler.synchronized { +CoarseGrainedSchedulerBackend.this.synchronized { fn } + } + } private[spark] object CoarseGrainedSchedulerBackend { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-26089][CORE] Handle corruption in large shuffle blocks
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 688b0c0 [SPARK-26089][CORE] Handle corruption in large shuffle blocks 688b0c0 is described below commit 688b0c01fac0db80f6473181673a89f1ce1be65b Author: ankurgupta AuthorDate: Tue Mar 12 14:27:44 2019 -0500 [SPARK-26089][CORE] Handle corruption in large shuffle blocks ## What changes were proposed in this pull request? SPARK-4105 added corruption detection in shuffle blocks but that was limited to blocks which are smaller than maxBytesInFlight/3. This commit adds upon that by adding corruption check for large blocks. There are two changes/improvements that are made in this commit: 1. Large blocks are checked upto maxBytesInFlight/3 size in a similar way as smaller blocks, so if a large block is corrupt in the starting, that block will be re-fetched and if that also fails, FetchFailureException will be thrown. 2. If large blocks are corrupt after size maxBytesInFlight/3, then any IOException thrown while reading the stream will be converted to FetchFailureException. This is slightly more aggressive than was originally intended but since the consumer of the stream may have already read some records and processed them, we can't just re-fetch the block, we need to fail the whole task. Additionally, we also thought about maybe adding a new type of TaskEndReason, which would re-try the task couple of times before failing the previous stage, but given the complexity involved in that solution we decided to not proceed in that direction. Thanks to squito for direction and support. ## How was this patch tested? Changed the junit test for big blocks to check for corruption. Closes #23453 from ankuriitg/ankurgupta/SPARK-26089. Authored-by: ankurgupta Signed-off-by: Imran Rashid --- .../org/apache/spark/internal/config/package.scala | 9 ++ .../spark/shuffle/BlockStoreShuffleReader.scala| 1 + .../storage/ShuffleBlockFetcherIterator.scala | 94 + .../main/scala/org/apache/spark/util/Utils.scala | 45 +++ .../storage/ShuffleBlockFetcherIteratorSuite.scala | 148 ++--- .../scala/org/apache/spark/util/UtilsSuite.scala | 54 +++- 6 files changed, 306 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index d6a359d..850d684 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -928,6 +928,15 @@ package object config { .booleanConf .createWithDefault(true) + private[spark] val SHUFFLE_DETECT_CORRUPT_MEMORY = +ConfigBuilder("spark.shuffle.detectCorrupt.useExtraMemory") + .doc("If enabled, part of a compressed/encrypted stream will be de-compressed/de-crypted " + +"by using extra memory to detect early corruption. Any IOException thrown will cause " + +"the task to be retried once and if it fails again with same exception, then " + +"FetchFailedException will be thrown to retry previous stage") + .booleanConf + .createWithDefault(false) + private[spark] val SHUFFLE_SYNC = ConfigBuilder("spark.shuffle.sync") .doc("Whether to force outstanding writes to disk.") diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index c5eefc7..c784371 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -55,6 +55,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT), + SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), readMetrics).toCompletionIterator val serializerInstance = dep.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index c75b209..c89d5cc 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{InputStream, IOException} +
[spark] branch branch-2.3 updated: [SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created TaskSet should learn about the finished partitions
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.3 by this push: new a1ca566 [SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created TaskSet should learn about the finished partitions a1ca566 is described below commit a1ca5663725c278b6e3785042348819a25496fe4 Author: wuyi AuthorDate: Thu Mar 7 12:30:37 2019 -0600 [SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created TaskSet should learn about the finished partitions ## What changes were proposed in this pull request? This is an optional solution for #22806 . #21131 firstly implement that a previous successful completed task from zombie TaskSetManager could also succeed the active TaskSetManager, which based on an assumption that an active TaskSetManager always exists for that stage when this happen. But that's not always true as an active TaskSetManager may haven't been created when a previous task succeed, and this is the reason why #22806 hit the issue. This pr extends #21131 's behavior by adding stageIdToFinishedPartitions into TaskSchedulerImpl, which recording the finished partition whenever a task(from zombie or active) succeed. Thus, a later created active TaskSetManager could also learn about the finished partition by looking into stageIdToFinishedPartitions and won't launch any duplicate tasks. ## How was this patch tested? Add. Closes #24007 from Ngone51/dev-23433-25250-branch-2.3. Authored-by: wuyi Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 -- .../apache/spark/scheduler/TaskSetManager.scala| 19 +++--- .../spark/scheduler/TaskSchedulerImplSuite.scala | 44 -- 3 files changed, 79 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index edf79aa..ef3ce87 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong import scala.collection.Set -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet} import scala.util.Random import org.apache.spark._ @@ -93,6 +93,9 @@ private[spark] class TaskSchedulerImpl( private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, TaskSetManager] val taskIdToExecutorId = new HashMap[Long, String] + // Protected by `this` + private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet] + @volatile private var hasReceivedTask = false @volatile private var hasLaunchedTask = false private val starvationTimer = new Timer(true) @@ -223,7 +226,20 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { -new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// only create a BitSet once for a certain stage since we only remove +// that stage when an active TaskSetManager succeed. +stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet) +val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// TaskSet got submitted by DAGScheduler may have some already completed +// tasks since DAGScheduler does not always know all the tasks that have +// been completed by other tasksets when completing a stage, so we mark +// those tasks as finished here to avoid launching duplicate tasks, while +// holding the TaskSchedulerImpl lock. +// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()` +stageIdToFinishedPartitions.get(taskSet.stageId).foreach { + finishedPartitions => finishedPartitions.foreach(tsm.markPartitionCompleted(_, None)) +} +tsm } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -696,19 +712,31 @@ private[spark] class TaskSchedulerImpl( } /** - * Marks the task has completed in all TaskSetManagers for the given stage. + * Marks the task has completed in all TaskSetManagers(active / zombie) for the given stage. * * After stage failure and retry, there may be multiple TaskSetManagers for the stage. * If an earlier attempt of a stage completes a task, we should ensure that the later attempts * do not also submit those same tasks. That also means that a task completion from an earlier * attempt can lead to the entire stage gett
[spark] branch branch-2.4 updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new 7df5aa6 [SPARK-27065][CORE] avoid more than one active task set managers for a stage 7df5aa6 is described below commit 7df5aa6b8a81478e25532064e1177c37027ec423 Author: Wenchen Fan AuthorDate: Wed Mar 6 12:00:33 2019 -0600 [SPARK-27065][CORE] avoid more than one active task set managers for a stage ## What changes were proposed in this pull request? This is another attempt to fix the more-than-one-active-task-set-managers bug. https://github.com/apache/spark/pull/17208 is the first attempt. It marks the TSM as zombie before sending a task completion event to DAGScheduler. This is necessary, because when the DAGScheduler gets the task completion event, and it's for the last partition, then the stage is finished. However, if it's a shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see the [code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule [...] This fix has a hole: Let's say a stage has 10 partitions and 2 task set managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is still active because he hasn't finished partition 10 yet. However, DAGScheduler gets task completion events for all the 10 partitions and thinks the stage is finished. Then the same problem occurs: DAGScheduler may resubmit the stage and cause more than one ac [...] https://github.com/apache/spark/pull/21131 fixed this hole by notifying all the task set managers when a task finishes. For the above case, TSM2 will know that partition 10 is already completed, so he can mark himself as zombie after partitions 1-9 are completed. However, #21131 still has a hole: TSM2 may be created after the task from TSM1 is completed. Then TSM2 can't get notified about the task completion, and leads to the more than one active TSM error. #22806 and #23871 are created to fix this hole. However the fix is complicated and there are still ongoing discussions. This PR proposes a simple fix, which can be easy to backport: mark all existing task set managers as zombie when trying to create a new task set manager. After this PR, #21131 is still necessary, to avoid launching unnecessary tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 ). #22806 and #23871 are its followups to fix the hole. ## How was this patch tested? existing tests. Closes #23927 from cloud-fan/scheduler. Authored-by: Wenchen Fan Signed-off-by: Imran Rashid (cherry picked from commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750) Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 - .../spark/scheduler/TaskSchedulerImplSuite.scala | 33 ++ 2 files changed, 35 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 4d45fd2..382a417 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -207,14 +207,20 @@ private[spark] class TaskSchedulerImpl( val stage = taskSet.stageId val stageTaskSets = taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, TaskSetManager]) - stageTaskSets(taskSet.stageAttemptId) = manager - val conflictingTaskSet = stageTaskSets.exists { case (_, ts) => -ts.taskSet != taskSet && !ts.isZombie - } - if (conflictingTaskSet) { -throw new IllegalStateException(s"more than one active taskSet for stage $stage:" + - s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}") + + // Mark all the existing TaskSetManagers of this stage as zombie, as we are adding a new one. + // This is necessary to handle a corner case. Let's say a stage has 10 partitions and has 2 + // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 + // and it completes. TSM2 finishes tasks for partition 1-9, and thinks he is still active + // because partition 10 is not completed yet. However, DAGScheduler gets task completion + // events for all the 10 partitions and thinks the stage is finished. If it's a shuffle stage + // and somehow it has missing map outputs, then DAGScheduler will resubmit it and create a + // TSM3 for it. As a stage can't have more than one active task s
[spark] branch branch-2.3 updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.3 by this push: new 877b8db [SPARK-27065][CORE] avoid more than one active task set managers for a stage 877b8db is described below commit 877b8db25b70ffb0793a619342e7b8edda712b31 Author: Wenchen Fan AuthorDate: Wed Mar 6 12:00:33 2019 -0600 [SPARK-27065][CORE] avoid more than one active task set managers for a stage ## What changes were proposed in this pull request? This is another attempt to fix the more-than-one-active-task-set-managers bug. https://github.com/apache/spark/pull/17208 is the first attempt. It marks the TSM as zombie before sending a task completion event to DAGScheduler. This is necessary, because when the DAGScheduler gets the task completion event, and it's for the last partition, then the stage is finished. However, if it's a shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see the [code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule [...] This fix has a hole: Let's say a stage has 10 partitions and 2 task set managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is still active because he hasn't finished partition 10 yet. However, DAGScheduler gets task completion events for all the 10 partitions and thinks the stage is finished. Then the same problem occurs: DAGScheduler may resubmit the stage and cause more than one ac [...] https://github.com/apache/spark/pull/21131 fixed this hole by notifying all the task set managers when a task finishes. For the above case, TSM2 will know that partition 10 is already completed, so he can mark himself as zombie after partitions 1-9 are completed. However, #21131 still has a hole: TSM2 may be created after the task from TSM1 is completed. Then TSM2 can't get notified about the task completion, and leads to the more than one active TSM error. #22806 and #23871 are created to fix this hole. However the fix is complicated and there are still ongoing discussions. This PR proposes a simple fix, which can be easy to backport: mark all existing task set managers as zombie when trying to create a new task set manager. After this PR, #21131 is still necessary, to avoid launching unnecessary tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 ). #22806 and #23871 are its followups to fix the hole. ## How was this patch tested? existing tests. Closes #23927 from cloud-fan/scheduler. Authored-by: Wenchen Fan Signed-off-by: Imran Rashid (cherry picked from commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750) Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 - .../spark/scheduler/TaskSchedulerImplSuite.scala | 33 ++ 2 files changed, 35 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 4edc6b2..edf79aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -185,14 +185,20 @@ private[spark] class TaskSchedulerImpl( val stage = taskSet.stageId val stageTaskSets = taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, TaskSetManager]) - stageTaskSets(taskSet.stageAttemptId) = manager - val conflictingTaskSet = stageTaskSets.exists { case (_, ts) => -ts.taskSet != taskSet && !ts.isZombie - } - if (conflictingTaskSet) { -throw new IllegalStateException(s"more than one active taskSet for stage $stage:" + - s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}") + + // Mark all the existing TaskSetManagers of this stage as zombie, as we are adding a new one. + // This is necessary to handle a corner case. Let's say a stage has 10 partitions and has 2 + // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 + // and it completes. TSM2 finishes tasks for partition 1-9, and thinks he is still active + // because partition 10 is not completed yet. However, DAGScheduler gets task completion + // events for all the 10 partitions and thinks the stage is finished. If it's a shuffle stage + // and somehow it has missing map outputs, then DAGScheduler will resubmit it and create a + // TSM3 for it. As a stage can't have more than one active task s
[spark] branch master updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new cb20fbc [SPARK-27065][CORE] avoid more than one active task set managers for a stage cb20fbc is described below commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750 Author: Wenchen Fan AuthorDate: Wed Mar 6 12:00:33 2019 -0600 [SPARK-27065][CORE] avoid more than one active task set managers for a stage ## What changes were proposed in this pull request? This is another attempt to fix the more-than-one-active-task-set-managers bug. https://github.com/apache/spark/pull/17208 is the first attempt. It marks the TSM as zombie before sending a task completion event to DAGScheduler. This is necessary, because when the DAGScheduler gets the task completion event, and it's for the last partition, then the stage is finished. However, if it's a shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see the [code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule [...] This fix has a hole: Let's say a stage has 10 partitions and 2 task set managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is still active because he hasn't finished partition 10 yet. However, DAGScheduler gets task completion events for all the 10 partitions and thinks the stage is finished. Then the same problem occurs: DAGScheduler may resubmit the stage and cause more than one ac [...] https://github.com/apache/spark/pull/21131 fixed this hole by notifying all the task set managers when a task finishes. For the above case, TSM2 will know that partition 10 is already completed, so he can mark himself as zombie after partitions 1-9 are completed. However, #21131 still has a hole: TSM2 may be created after the task from TSM1 is completed. Then TSM2 can't get notified about the task completion, and leads to the more than one active TSM error. #22806 and #23871 are created to fix this hole. However the fix is complicated and there are still ongoing discussions. This PR proposes a simple fix, which can be easy to backport: mark all existing task set managers as zombie when trying to create a new task set manager. After this PR, #21131 is still necessary, to avoid launching unnecessary tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 ). #22806 and #23871 are its followups to fix the hole. ## How was this patch tested? existing tests. Closes #23927 from cloud-fan/scheduler. Authored-by: Wenchen Fan Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 - .../spark/scheduler/TaskSchedulerImplSuite.scala | 33 ++ 2 files changed, 35 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index c23b201..db0fbfe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -215,14 +215,20 @@ private[spark] class TaskSchedulerImpl( val stage = taskSet.stageId val stageTaskSets = taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, TaskSetManager]) - stageTaskSets(taskSet.stageAttemptId) = manager - val conflictingTaskSet = stageTaskSets.exists { case (_, ts) => -ts.taskSet != taskSet && !ts.isZombie - } - if (conflictingTaskSet) { -throw new IllegalStateException(s"more than one active taskSet for stage $stage:" + - s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}") + + // Mark all the existing TaskSetManagers of this stage as zombie, as we are adding a new one. + // This is necessary to handle a corner case. Let's say a stage has 10 partitions and has 2 + // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 + // and it completes. TSM2 finishes tasks for partition 1-9, and thinks he is still active + // because partition 10 is not completed yet. However, DAGScheduler gets task completion + // events for all the 10 partitions and thinks the stage is finished. If it's a shuffle stage + // and somehow it has missing map outputs, then DAGScheduler will resubmit it and create a + // TSM3 for it. As a stage can't have more than one active task set managers, we must mark + // TSM2 as zombie (it actually is). + stageTaskSets.foreach { ca
[spark] branch branch-2.4 updated: [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new db86ccb [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions db86ccb is described below commit db86ccb11821231d85b727fb889dec1d58b39e4d Author: wuyi AuthorDate: Wed Mar 6 11:53:07 2019 -0600 [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions ## What changes were proposed in this pull request? This is an optional solution for #22806 . #21131 firstly implement that a previous successful completed task from zombie TaskSetManager could also succeed the active TaskSetManager, which based on an assumption that an active TaskSetManager always exists for that stage when this happen. But that's not always true as an active TaskSetManager may haven't been created when a previous task succeed, and this is the reason why #22806 hit the issue. This pr extends #21131 's behavior by adding `stageIdToFinishedPartitions` into TaskSchedulerImpl, which recording the finished partition whenever a task(from zombie or active) succeed. Thus, a later created active TaskSetManager could also learn about the finished partition by looking into `stageIdToFinishedPartitions ` and won't launch any duplicate tasks. ## How was this patch tested? Add. Closes #23871 from Ngone51/dev-23433-25250. Lead-authored-by: wuyi Co-authored-by: Ngone51 Signed-off-by: Imran Rashid (cherry picked from commit e5c61436a5720f13eb6d530ebf80635522bd64c6) Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 -- .../apache/spark/scheduler/TaskSetManager.scala| 19 +++--- .../spark/scheduler/TaskSchedulerImplSuite.scala | 44 -- 3 files changed, 79 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 24d77f8..4d45fd2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong import scala.collection.Set -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet} import scala.util.Random import org.apache.spark._ @@ -94,6 +94,9 @@ private[spark] class TaskSchedulerImpl( private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, TaskSetManager] val taskIdToExecutorId = new HashMap[Long, String] + // Protected by `this` + private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet] + @volatile private var hasReceivedTask = false @volatile private var hasLaunchedTask = false private val starvationTimer = new Timer(true) @@ -236,7 +239,20 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { -new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// only create a BitSet once for a certain stage since we only remove +// that stage when an active TaskSetManager succeed. +stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet) +val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// TaskSet got submitted by DAGScheduler may have some already completed +// tasks since DAGScheduler does not always know all the tasks that have +// been completed by other tasksets when completing a stage, so we mark +// those tasks as finished here to avoid launching duplicate tasks, while +// holding the TaskSchedulerImpl lock. +// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()` +stageIdToFinishedPartitions.get(taskSet.stageId).foreach { + finishedPartitions => finishedPartitions.foreach(tsm.markPartitionCompleted(_, None)) +} +tsm } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -833,19 +849,31 @@ private[spark] class TaskSchedulerImpl( } /** - * Marks the task has completed in all TaskSetManagers for the given stage. + * Marks the task has completed in all TaskSetManagers(active / zombie) for the given stage. * * After stage failure and retry, there may be multiple TaskSetManagers for the stage. * If an earlier attempt of a stage completes a task, we should ensure that the later attempts * do not also submit those same ta
[spark] branch master updated: [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e5c6143 [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions e5c6143 is described below commit e5c61436a5720f13eb6d530ebf80635522bd64c6 Author: wuyi AuthorDate: Wed Mar 6 11:53:07 2019 -0600 [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions ## What changes were proposed in this pull request? This is an optional solution for #22806 . #21131 firstly implement that a previous successful completed task from zombie TaskSetManager could also succeed the active TaskSetManager, which based on an assumption that an active TaskSetManager always exists for that stage when this happen. But that's not always true as an active TaskSetManager may haven't been created when a previous task succeed, and this is the reason why #22806 hit the issue. This pr extends #21131 's behavior by adding `stageIdToFinishedPartitions` into TaskSchedulerImpl, which recording the finished partition whenever a task(from zombie or active) succeed. Thus, a later created active TaskSetManager could also learn about the finished partition by looking into `stageIdToFinishedPartitions ` and won't launch any duplicate tasks. ## How was this patch tested? Add. Closes #23871 from Ngone51/dev-23433-25250. Lead-authored-by: wuyi Co-authored-by: Ngone51 Signed-off-by: Imran Rashid --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 -- .../apache/spark/scheduler/TaskSetManager.scala| 19 +++--- .../spark/scheduler/TaskSchedulerImplSuite.scala | 44 -- 3 files changed, 79 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 3f23bfe..c23b201 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong import scala.collection.Set -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet} import scala.util.Random import org.apache.spark._ @@ -102,6 +102,9 @@ private[spark] class TaskSchedulerImpl( // Protected by `this` val taskIdToExecutorId = new HashMap[Long, String] + // Protected by `this` + private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet] + @volatile private var hasReceivedTask = false @volatile private var hasLaunchedTask = false private val starvationTimer = new Timer(true) @@ -244,7 +247,20 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { -new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// only create a BitSet once for a certain stage since we only remove +// that stage when an active TaskSetManager succeed. +stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet) +val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) +// TaskSet got submitted by DAGScheduler may have some already completed +// tasks since DAGScheduler does not always know all the tasks that have +// been completed by other tasksets when completing a stage, so we mark +// those tasks as finished here to avoid launching duplicate tasks, while +// holding the TaskSchedulerImpl lock. +// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()` +stageIdToFinishedPartitions.get(taskSet.stageId).foreach { + finishedPartitions => finishedPartitions.foreach(tsm.markPartitionCompleted(_, None)) +} +tsm } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -847,19 +863,31 @@ private[spark] class TaskSchedulerImpl( } /** - * Marks the task has completed in all TaskSetManagers for the given stage. + * Marks the task has completed in all TaskSetManagers(active / zombie) for the given stage. * * After stage failure and retry, there may be multiple TaskSetManagers for the stage. * If an earlier attempt of a stage completes a task, we should ensure that the later attempts * do not also submit those same tasks. That also means that a task completion from an earlier * attempt can lead to the entire stage getting marked as successful. + * And there is also the possibil
[spark] branch master updated: [SPARK-25865][CORE] Add GC information to ExecutorMetrics
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e5c502c [SPARK-25865][CORE] Add GC information to ExecutorMetrics e5c502c is described below commit e5c502c596563dce8eb58f86e42c1aea2c51ed17 Author: LantaoJin AuthorDate: Mon Mar 4 14:26:02 2019 -0600 [SPARK-25865][CORE] Add GC information to ExecutorMetrics ## What changes were proposed in this pull request? Only memory usage without GC information could not help us to determinate the proper settings of memory. We need the GC metrics about frequency of major & minor GC. For example, two cases, their configured memory for executor are all 10GB and their usages are all near 10GB. So should we increase or decrease the configured memory for them? This metrics may be helpful. We can increase configured memory for the first one if it has very frequency major GC and decrease the second one if on [...] GC metrics are only useful in entire lifetime of executors instead of separated stages. ## How was this patch tested? Adding UT. Closes #22874 from LantaoJin/SPARK-25865. Authored-by: LantaoJin Signed-off-by: Imran Rashid --- .../org/apache/spark/internal/config/package.scala | 19 ++ .../apache/spark/metrics/ExecutorMetricType.scala | 63 +- .../application_list_json_expectation.json | 18 ++ .../completed_app_list_json_expectation.json | 18 ++ ...arbage_collection_metrics_json_expectation.json | 122 ...ist_with_executor_metrics_json_expectation.json | 30 ++- ...utor_process_tree_metrics_json_expectation.json | 12 +- .../limit_app_list_json_expectation.json | 33 ++-- .../minDate_app_list_json_expectation.json | 18 ++ .../minEndDate_app_list_json_expectation.json | 18 ++ .../spark-events/application_1536831636016_59384_1 | 214 + .../spark/deploy/history/HistoryServerSuite.scala | 2 + .../scheduler/EventLoggingListenerSuite.scala | 36 ++-- .../org/apache/spark/util/JsonProtocolSuite.scala | 18 +- dev/.rat-excludes | 1 + 15 files changed, 576 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3ef6cba..d6a359d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.internal import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.metrics.GarbageCollectionMetrics import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} @@ -114,6 +115,24 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS = +ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors") + .doc("Names of supported young generation garbage collector. A name usually is " + +" the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " + +s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + + private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS = +ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors") + .doc("Names of supported old generation garbage collector. A name usually is " + +"the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " + +s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 704b36d..6d8e9a8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,9 +19,12 @@ package org.apache.spark.metrics import
[spark] branch master updated: [SPARK-26688][YARN] Provide configuration of initially blacklisted YARN nodes
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new caceaec [SPARK-26688][YARN] Provide configuration of initially blacklisted YARN nodes caceaec is described below commit caceaec93203edaea1d521b88e82ef67094cdea9 Author: “attilapiros” AuthorDate: Mon Mar 4 14:14:20 2019 -0600 [SPARK-26688][YARN] Provide configuration of initially blacklisted YARN nodes ## What changes were proposed in this pull request? Introducing new config for initially blacklisted YARN nodes. ## How was this patch tested? With existing and a new unit test. Closes #23616 from attilapiros/SPARK-26688. Lead-authored-by: “attilapiros” Co-authored-by: Attila Zsolt Piros <2017933+attilapi...@users.noreply.github.com> Signed-off-by: Imran Rashid --- docs/running-on-yarn.md| 7 + .../yarn/YarnAllocatorBlacklistTracker.scala | 4 ++- .../org/apache/spark/deploy/yarn/config.scala | 6 .../yarn/YarnAllocatorBlacklistTrackerSuite.scala | 33 -- 4 files changed, 41 insertions(+), 9 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 6ee4b3d..261f7e3 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -463,6 +463,13 @@ To use a custom metrics.properties for the application master and executors, upd + spark.yarn.exclude.nodes + (none) + + Comma-separated list of YARN node names which are excluded from resource allocation. + + + spark.yarn.metrics.namespace (none) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala index 268976b..fa8c961 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala @@ -56,6 +56,8 @@ private[spark] class YarnAllocatorBlacklistTracker( private val maxFailuresPerHost = sparkConf.get(MAX_FAILED_EXEC_PER_NODE) + private val excludeNodes = sparkConf.get(YARN_EXCLUDE_NODES).toSet + private val allocatorBlacklist = new HashMap[String, Long]() private var currentBlacklistedYarnNodes = Set.empty[String] @@ -105,7 +107,7 @@ private[spark] class YarnAllocatorBlacklistTracker( private def refreshBlacklistedNodes(): Unit = { removeExpiredYarnBlacklistedNodes() -val allBlacklistedNodes = schedulerBlacklist ++ allocatorBlacklist.keySet +val allBlacklistedNodes = excludeNodes ++ schedulerBlacklist ++ allocatorBlacklist.keySet synchronizeBlacklistedNodeWithYarn(allBlacklistedNodes) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 5f8739b..4c187b2 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -310,6 +310,12 @@ package object config { .booleanConf .createWithDefault(false) + /* Initially blacklisted YARN nodes. */ + private[spark] val YARN_EXCLUDE_NODES = ConfigBuilder("spark.yarn.exclude.nodes") + .stringConf + .toSequence + .createWithDefault(Nil) + private[yarn] val YARN_EXECUTOR_RESOURCE_TYPES_PREFIX = "spark.yarn.executor.resource." private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala index 259d758..c07a4ac 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala @@ -25,7 +25,7 @@ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.yarn.config.YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED +import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED} import org.apache.spark.internal.config.{BLACKLIST_TIMEOUT_CONF, MAX_FAILED_EXEC_PER_NODE} import org.apache.spark.util.ManualClock @@ -35,27 +35,31 @@ class YarnAllocat
[spark] branch master updated: [SPARK-24522][UI] Create filter to apply HTTP security checks consistently.
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2783e4c [SPARK-24522][UI] Create filter to apply HTTP security checks consistently. 2783e4c is described below commit 2783e4c45f55f4fc87748d1c4a454bfdf3024156 Author: Marcelo Vanzin AuthorDate: Tue Jan 8 11:25:33 2019 -0600 [SPARK-24522][UI] Create filter to apply HTTP security checks consistently. Currently there is code scattered in a bunch of places to do different things related to HTTP security, such as access control, setting security-related headers, and filtering out bad content. This makes it really easy to miss these things when writing new UI code. This change creates a new filter that does all of those things, and makes sure that all servlet handlers that are attached to the UI get the new filter and any user-defined filters consistently. The extent of the actual features should be the same as before. The new filter is added at the end of the filter chain, because authentication is done by custom filters and thus needs to happen first. This means that custom filters see unfiltered HTTP requests - which is actually the current behavior anyway. As a side-effect of some of the code refactoring, handlers added after the initial set also get wrapped with a GzipHandler, which didn't happen before. Tested with added unit tests and in a history server with SPNEGO auth configured. Closes #23302 from vanzin/SPARK-24522. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../apache/spark/deploy/history/HistoryPage.scala | 5 +- .../spark/deploy/history/HistoryServer.scala | 8 +- .../spark/deploy/master/ui/ApplicationPage.scala | 3 +- .../apache/spark/deploy/master/ui/MasterPage.scala | 6 +- .../apache/spark/deploy/worker/ui/LogPage.scala| 28 ++-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 1 - .../apache/spark/metrics/sink/MetricsServlet.scala | 2 +- .../spark/status/api/v1/SecurityFilter.scala | 36 - .../org/apache/spark/ui/HttpSecurityFilter.scala | 116 +++ .../scala/org/apache/spark/ui/JettyUtils.scala | 154 +--- .../main/scala/org/apache/spark/ui/UIUtils.scala | 21 --- .../src/main/scala/org/apache/spark/ui/WebUI.scala | 15 +- .../spark/ui/exec/ExecutorThreadDumpPage.scala | 4 +- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 16 +-- .../scala/org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../scala/org/apache/spark/ui/jobs/JobsTab.scala | 4 +- .../scala/org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 19 ++- .../org/apache/spark/ui/jobs/StageTable.scala | 15 +- .../scala/org/apache/spark/ui/jobs/StagesTab.scala | 4 +- .../org/apache/spark/ui/storage/RDDPage.scala | 11 +- .../apache/spark/ui/HttpSecurityFilterSuite.scala | 157 + .../test/scala/org/apache/spark/ui/UISuite.scala | 147 +-- .../scala/org/apache/spark/ui/UIUtilsSuite.scala | 39 - .../apache/spark/deploy/mesos/ui/DriverPage.scala | 3 +- .../scheduler/cluster/YarnSchedulerBackend.scala | 35 - .../cluster/YarnSchedulerBackendSuite.scala| 59 +++- .../spark/sql/execution/ui/AllExecutionsPage.scala | 19 +-- .../spark/sql/execution/ui/ExecutionPage.scala | 3 +- .../thriftserver/ui/ThriftServerSessionPage.scala | 3 +- .../org/apache/spark/streaming/ui/BatchPage.scala | 8 +- 31 files changed, 609 insertions(+), 338 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 00ca4ef..7a8ab7f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -27,9 +27,8 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { -// stripXSS is called first to remove suspicious characters used in XSS attacks -val requestedIncomplete = - Option(UIUtils.stripXSS(request.getParameter("showIncomplete"))).getOrElse("false").toBoolean +val requestedIncomplete = Option(request.getParameter("showIncomplete")) + .getOrElse("false").toBoolean val displayApplications = parent.getApplicationList() .exists(isApplicationCompleted(_) != requestedIncomplete) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apa
[spark] branch master updated: [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens.
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 669e8a1 [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens. 669e8a1 is described below commit 669e8a155987995a1a5d49a96b88c05f39e41723 Author: Marcelo Vanzin AuthorDate: Mon Jan 7 14:40:08 2019 -0600 [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens. This change modifies the behavior of the delegation token code when running on YARN, so that the driver controls the renewal, in both client and cluster mode. For that, a few different things were changed: * The AM code only runs code that needs DTs when DTs are available. In a way, this restores the AM behavior to what it was pre-SPARK-23361, but keeping the fix added in that bug. Basically, all the AM code is run in a "UGI.doAs()" block; but code that needs to talk to HDFS (basically the distributed cache handling code) was delayed to the point where the driver is up and running, and thus when valid delegation tokens are available. * SparkSubmit / ApplicationMaster now handle user login, not the token manager. The previous AM code was relying on the token manager to keep the user logged in when keytabs are used. This required some odd APIs in the token manager and the AM so that the right UGI was exposed and used in the right places. After this change, the logged in user is handled separately from the token manager, so the API was cleaned up, and, as explained above, the whole AM runs under the logged in user, which also helps with simplifying some more code. * Distributed cache configs are sent separately to the AM. Because of the delayed initialization of the cached resources in the AM, it became easier to write the cache config to a separate properties file instead of bundling it with the rest of the Spark config. This also avoids having to modify the SparkConf to hide things from the UI. * Finally, the AM doesn't manage the token manager anymore. The above changes allow the token manager to be completely handled by the driver's scheduler backend code also in YARN mode (whether client or cluster), making it similar to other RMs. To maintain the fix added in SPARK-23361 also in client mode, the AM now sends an extra message to the driver on initialization to fetch delegation tokens; and although it might not really be needed, the driver also keeps the running AM updated when new tokens are created. Tested in a kerberized cluster with the same tests used to validate SPARK-23361, in both client and cluster mode. Also tested with a non-kerberized cluster. Closes #23338 from vanzin/SPARK-25689. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../security/HadoopDelegationTokenManager.scala| 110 ++--- .../security/HiveDelegationTokenProvider.scala | 16 ++- .../cluster/CoarseGrainedClusterMessage.scala | 3 + .../cluster/CoarseGrainedSchedulerBackend.scala| 40 -- .../HadoopDelegationTokenManagerSuite.scala| 8 +- .../features/KerberosConfDriverFeatureStep.scala | 2 +- .../k8s/KubernetesClusterSchedulerBackend.scala| 7 +- .../mesos/MesosCoarseGrainedSchedulerBackend.scala | 7 +- .../spark/deploy/yarn/ApplicationMaster.scala | 135 ++--- .../deploy/yarn/ApplicationMasterArguments.scala | 5 + .../org/apache/spark/deploy/yarn/Client.scala | 100 --- .../apache/spark/deploy/yarn/YarnRMClient.scala| 8 +- .../org/apache/spark/deploy/yarn/config.scala | 10 -- .../YARNHadoopDelegationTokenManager.scala | 7 +- .../cluster/YarnClientSchedulerBackend.scala | 6 + .../scheduler/cluster/YarnSchedulerBackend.scala | 17 ++- .../YARNHadoopDelegationTokenManagerSuite.scala| 2 +- 17 files changed, 246 insertions(+), 237 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index f7e3dde..d97857a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -21,7 +21,6 @@ import java.io.File import java.net.URI import java.security.PrivilegedExceptionAction import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicReference import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -39,32 +38,24 @@ import org.apache.spark.util.ThreadUtils /**
[spark] branch master updated: [SPARK-24958][CORE] Add memory from procfs to executor metrics.
This is an automated email from the ASF dual-hosted git repository. irashid pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 90c77ea [SPARK-24958][CORE] Add memory from procfs to executor metrics. 90c77ea is described below commit 90c77ea3132d0b7a12c316bd42fb8d0f59bee253 Author: Reza Safi AuthorDate: Mon Dec 10 11:14:11 2018 -0600 [SPARK-24958][CORE] Add memory from procfs to executor metrics. This adds the entire memory used by spark’s executor (as measured by procfs) to the executor metrics. The memory usage is collected from the entire process tree under the executor. The metrics are subdivided into memory used by java, by python, and by other processes, to aid users in diagnosing the source of high memory usage. The additional metrics are sent to the driver in heartbeats, using the mechanism introduced by SPARK-23429. This also slightly extends that approach to allow one ExecutorMetricType to collect multiple metrics. Added unit tests and also tested on a live cluster. Closes #22612 from rezasafi/ptreememory2. Authored-by: Reza Safi Signed-off-by: Imran Rashid --- .../main/scala/org/apache/spark/Heartbeater.scala | 11 +- .../apache/spark/executor/ExecutorMetrics.scala| 23 +-- .../spark/executor/ProcfsMetricsGetter.scala | 228 + .../org/apache/spark/internal/config/package.scala | 5 + .../apache/spark/metrics/ExecutorMetricType.scala | 74 ++- .../scala/org/apache/spark/status/api/v1/api.scala | 6 +- .../scala/org/apache/spark/util/JsonProtocol.scala | 16 +- .../application_list_json_expectation.json | 15 ++ .../completed_app_list_json_expectation.json | 15 ++ ...ist_with_executor_metrics_json_expectation.json | 40 +++- ...utor_process_tree_metrics_json_expectation.json | 98 + .../limit_app_list_json_expectation.json | 30 +-- .../minDate_app_list_json_expectation.json | 15 ++ .../minEndDate_app_list_json_expectation.json | 15 ++ core/src/test/resources/ProcfsMetrics/22763/stat | 1 + core/src/test/resources/ProcfsMetrics/26109/stat | 1 + .../spark-events/application_1538416563558_0014| 190 + .../spark/deploy/history/HistoryServerSuite.scala | 3 + .../spark/executor/ProcfsMetricsGetterSuite.scala | 41 .../scheduler/EventLoggingListenerSuite.scala | 85 +--- .../spark/status/AppStatusListenerSuite.scala | 74 --- .../org/apache/spark/util/JsonProtocolSuite.scala | 46 +++-- dev/.rat-excludes | 2 + 23 files changed, 901 insertions(+), 133 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 84091ee..1012755 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,10 +61,17 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by ExecutorMetricType.values + * determined by ExecutorMetricType.metricToOffset */ def getCurrentMetrics(): ExecutorMetrics = { -val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray + +val metrics = new Array[Long](ExecutorMetricType.numMetrics) +var offset = 0 +ExecutorMetricType.metricGetters.foreach { metric => + val newMetrics = metric.getMetricValues(memoryManager) + Array.copy(newMetrics, 0, metrics, offset, newMetrics.size) + offset += newMetrics.length +} new ExecutorMetrics(metrics) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 1befd27..f19ac81 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -27,17 +27,15 @@ import org.apache.spark.metrics.ExecutorMetricType */ @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - - // Metrics are indexed by ExecutorMetricType.values - private val metrics = new Array[Long](ExecutorMetricType.values.length) - + // Metrics are indexed by ExecutorMetricType.metricToOffset + private val metrics = new Array[Long](ExecutorMetricType.numMetrics) // the first element is initialized to -1, indicating that the values for the array // haven't been set yet. metrics(0) = -1 - /** Returns the value for the specified metricType. */ - def getMetricValue(metricType: ExecutorMetricType): Long = { -metrics(ExecutorMetricType.metricIdxMap(metricType)) + /** Returns the value for the specified metric. */ + def getMetricValue(metricName: Str
spark git commit: [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer
Repository: spark Updated Branches: refs/heads/master cb368f2c2 -> 59741887e [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer ## What changes were proposed in this pull request? In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling `getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`. Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just get this `ManagedBuffer` and use its `InputStream`. When reading a remote cache block from disk, this reduces heap memory usage significantly. Retain `getRemoteBytes` for other callers. ## How was this patch tested? Imran Rashid wrote an application (https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala), that among other things, tests reading remote cache blocks. I ran this application, using 2500MB blocks, to test reading a cache block on disk. Without this change, with `--executor-memory 5g`, the test fails with `java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes with `--executor-memory 2g`. I also ran the unit tests in core. In particular, `DistributedSuite` has a set of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has several tests that call `getRemoteBytes`; I left these unchanged, so `getRemoteBytes` still gets exercised. Closes #23058 from wypoon/SPARK-25905. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/59741887 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/59741887 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/59741887 Branch: refs/heads/master Commit: 59741887e272be92ebd6e61783f99f7d8fc05456 Parents: cb368f2 Author: Wing Yew Poon Authored: Thu Nov 29 14:56:34 2018 -0600 Committer: Imran Rashid Committed: Thu Nov 29 14:56:34 2018 -0600 -- .../org/apache/spark/storage/BlockManager.scala | 43 .../spark/util/io/ChunkedByteBuffer.scala | 2 - .../org/apache/spark/DistributedSuite.scala | 2 +- 3 files changed, 28 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/59741887/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1b61729..1dfbc6e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -692,9 +692,9 @@ private[spark] class BlockManager( */ private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val ct = implicitly[ClassTag[T]] -getRemoteBytes(blockId).map { data => +getRemoteManagedBuffer(blockId).map { data => val values = -serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct) +serializerManager.dataDeserializeStream(blockId, data.createInputStream())(ct) new BlockResult(values, DataReadMethod.Network, data.size) } } @@ -717,13 +717,9 @@ private[spark] class BlockManager( } /** - * Get block from remote block managers as serialized bytes. + * Get block from remote block managers as a ManagedBuffer. */ - def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { -// TODO SPARK-25905 if we change this method to return the ManagedBuffer, then getRemoteValues -// could just use the inputStream on the temp file, rather than reading the file into memory. -// Until then, replication can cause the process to use too much memory and get killed -// even though we've read the data to disk. + private def getRemoteManagedBuffer(blockId: BlockId): Option[ManagedBuffer] = { logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") var runningFailureCount = 0 @@ -788,14 +784,13 @@ private[spark] class BlockManager( } if (data != null) { -// SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to -// ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if -// new path is stable. -if (remoteReadNioBufferConversion) { - return Some(new ChunkedByteBuffer(data.nioByteBuffer())) -} else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data)) -} +// If the ManagedBuffer is a BlockManagerManagedBuffer, the disposal of the +
spark git commit: [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize
Repository: spark Updated Branches: refs/heads/master c34c42234 -> ab61ddb34 [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize ## What changes were proposed in this pull request? Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize. This way long authorization field does not lead to HTTP 413. ## How was this patch tested? Manually with curl (which version must be at least 7.55). With the original default value (8k limit): ```bash # Starting history server with default requestHeaderSize $ ./sbin/start-history-server.sh starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # Creating huge header $ echo -n "X-Custom-Header: " > cookie $ printf 'A%.0s' {1..9500} >> cookie # HTTP GET with huge header fails with 431 $ curl -H cookie http://458apiros-MBP.lan:18080/ Bad Message 431reason: Request Header Fields Too Large # The log contains the error $ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out 18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192 ``` After: ```bash # Creating the history properties file with the increased requestHeaderSize $ echo spark.ui.requestHeaderSize=1 > history.properties # Starting Spark History Server with the settings $ ./sbin/start-history-server.sh --properties-file history.properties starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response) $ curl -H cookie http://458apiros-MBP.lan:18080/ ... History Server ... ``` Closes #23090 from attilapiros/JettyHeaderSize. Authored-by: âattilapirosâ Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ab61ddb3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ab61ddb3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ab61ddb3 Branch: refs/heads/master Commit: ab61ddb34d58ab5701191c8fd3a24a62f6ebf37b Parents: c34c422 Author: âattilapirosâ Authored: Tue Nov 20 08:56:22 2018 -0600 Committer: Imran Rashid Committed: Tue Nov 20 08:56:22 2018 -0600 -- .../scala/org/apache/spark/internal/config/package.scala | 6 ++ core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 -- docs/configuration.md| 8 3 files changed, 18 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ab61ddb3/core/src/main/scala/org/apache/spark/internal/config/package.scala -- diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ab2b872..9cc48f6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -570,6 +570,12 @@ package object config { .stringConf .createOptional + private[spark] val UI_REQUEST_HEADER_SIZE = +ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf http://git-wip-us.apache.org/repos/asf/spark/blob/ab61ddb3/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 52a9551..316af9b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -356,13 +356,15 @@ private[spark] object JettyUtils extends Logging { (connector, connector.getLocalPort()) } + val httpConfig = new HttpConfiguration() + httpConfig.setRequestHeaderSize(conf.get(UI_REQUEST_HEADER_SIZE).toInt) // If SSL is configured, create the secure connector first. val securePort = sslOptions.createJettySslContextFactory().map { factory => val securePort = sslOptions.port.getOrElse(if (port > 0) Utils.userPort(port, 400) else 0)
spark git commit: [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize
Repository: spark Updated Branches: refs/heads/branch-2.4 096e0d8f0 -> c28a27a25 [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize ## What changes were proposed in this pull request? Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize. This way long authorization field does not lead to HTTP 413. ## How was this patch tested? Manually with curl (which version must be at least 7.55). With the original default value (8k limit): ```bash # Starting history server with default requestHeaderSize $ ./sbin/start-history-server.sh starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # Creating huge header $ echo -n "X-Custom-Header: " > cookie $ printf 'A%.0s' {1..9500} >> cookie # HTTP GET with huge header fails with 431 $ curl -H cookie http://458apiros-MBP.lan:18080/ Bad Message 431reason: Request Header Fields Too Large # The log contains the error $ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out 18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192 ``` After: ```bash # Creating the history properties file with the increased requestHeaderSize $ echo spark.ui.requestHeaderSize=1 > history.properties # Starting Spark History Server with the settings $ ./sbin/start-history-server.sh --properties-file history.properties starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response) $ curl -H cookie http://458apiros-MBP.lan:18080/ ... History Server ... ``` Closes #23090 from attilapiros/JettyHeaderSize. Authored-by: âattilapirosâ Signed-off-by: Imran Rashid (cherry picked from commit ab61ddb34d58ab5701191c8fd3a24a62f6ebf37b) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c28a27a2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c28a27a2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c28a27a2 Branch: refs/heads/branch-2.4 Commit: c28a27a2546ebbe0c001662126625638fcbb1100 Parents: 096e0d8 Author: âattilapirosâ Authored: Tue Nov 20 08:56:22 2018 -0600 Committer: Imran Rashid Committed: Tue Nov 20 08:56:39 2018 -0600 -- .../scala/org/apache/spark/internal/config/package.scala | 6 ++ core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 -- docs/configuration.md| 8 3 files changed, 18 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c28a27a2/core/src/main/scala/org/apache/spark/internal/config/package.scala -- diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index bde0995..3b3c45f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -528,6 +528,12 @@ package object config { .stringConf .createOptional + private[spark] val UI_REQUEST_HEADER_SIZE = +ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf http://git-wip-us.apache.org/repos/asf/spark/blob/c28a27a2/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 52a9551..316af9b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -356,13 +356,15 @@ private[spark] object JettyUtils extends Logging { (connector, connector.getLocalPort()) } + val httpConfig = new HttpConfiguration() + httpConfig.setRequestHeaderSize(conf.get(UI_REQUEST_HEADER_SIZE).toInt) // If SSL is configured, create the secure connector first. val securePort = sslOptions.createJettySslContextFactory().map { factory =>
spark git commit: [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue
Repository: spark Updated Branches: refs/heads/branch-2.4 947462f5a -> 47a668c2f [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big. This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail. Closes #22818 from squito/SPARK-25827. Authored-by: Imran Rashid Signed-off-by: Imran Rashid (cherry picked from commit 8fbc1830f962c446b915d0d8ff2b13c5c75d22fc) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/47a668c2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/47a668c2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/47a668c2 Branch: refs/heads/branch-2.4 Commit: 47a668c2f03d77078259531ddaccf80b001a8b5c Parents: 947462f Author: Imran Rashid Authored: Wed Nov 7 13:18:52 2018 +0100 Committer: Imran Rashid Committed: Thu Nov 8 15:49:19 2018 +0100 -- .../org/apache/spark/internal/config/package.scala | 17 ++--- .../scala/org/apache/spark/storage/DiskStore.scala | 6 -- .../apache/spark/storage/memory/MemoryStore.scala | 7 --- .../apache/spark/util/io/ChunkedByteBuffer.scala | 2 +- .../org/apache/spark/mllib/linalg/Matrices.scala | 13 +++-- .../org/apache/spark/sql/internal/SQLConf.scala| 6 +++--- .../main/scala/org/apache/spark/sql/Dataset.scala | 7 +++ 7 files changed, 32 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/47a668c2/core/src/main/scala/org/apache/spark/internal/config/package.scala -- diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5836d27..e2162db 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -387,8 +387,9 @@ package object config { .internal() .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) - .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" + -" ChunkedByteBuffer should not larger than Int.MaxValue.") + .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, +"The chunk size during writing out the bytes of" + +" ChunkedByteBuffer should not larger than Int.MaxValue - 15.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -459,8 +460,9 @@ package object config { "otherwise specified. These buffers reduce the number of disk seeks and system calls " + "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, -s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, +s"The file buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = @@ -468,8 +470,9 @@ package object config { .doc("The file system for this buffer size after each partition " + "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, -s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, +s"The buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = @@ -580,7 +583,7 @@ package object config { .internal() .doc("For testing only, controls the size of chunks when memory mapping a file") .bytesConf(ByteUnit.BYTE) - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) private[spark] val BARRIER_SYNC_TIMEOUT = ConfigBuilder("spark.barrier.sync.timeout") http://git-wip-us.apache.org/repos/asf/spark/blob/47a668c2/core/src/main/scala/org/apache/spark/storage/DiskStore.scala -- diff --git
spark git commit: [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue
Repository: spark Updated Branches: refs/heads/master 9e9fa2f69 -> 8fbc1830f [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big. This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail. Closes #22818 from squito/SPARK-25827. Authored-by: Imran Rashid Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8fbc1830 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8fbc1830 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8fbc1830 Branch: refs/heads/master Commit: 8fbc1830f962c446b915d0d8ff2b13c5c75d22fc Parents: 9e9fa2f Author: Imran Rashid Authored: Wed Nov 7 13:18:52 2018 +0100 Committer: Imran Rashid Committed: Wed Nov 7 13:18:52 2018 +0100 -- .../org/apache/spark/internal/config/package.scala | 17 ++--- .../scala/org/apache/spark/storage/DiskStore.scala | 6 -- .../apache/spark/storage/memory/MemoryStore.scala | 7 --- .../apache/spark/util/io/ChunkedByteBuffer.scala | 2 +- .../org/apache/spark/mllib/linalg/Matrices.scala | 13 +++-- .../org/apache/spark/sql/internal/SQLConf.scala| 6 +++--- .../main/scala/org/apache/spark/sql/Dataset.scala | 7 +++ 7 files changed, 32 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8fbc1830/core/src/main/scala/org/apache/spark/internal/config/package.scala -- diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 2b3ba3c..d346013 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -416,8 +416,9 @@ package object config { .internal() .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) - .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" + -" ChunkedByteBuffer should not larger than Int.MaxValue.") + .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, +"The chunk size during writing out the bytes of" + +" ChunkedByteBuffer should not larger than Int.MaxValue - 15.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -488,8 +489,9 @@ package object config { "otherwise specified. These buffers reduce the number of disk seeks and system calls " + "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, -s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, +s"The file buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = @@ -497,8 +499,9 @@ package object config { .doc("The file system for this buffer size after each partition " + "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, -s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, +s"The buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = @@ -610,7 +613,7 @@ package object config { .internal() .doc("For testing only, controls the size of chunks when memory mapping a file") .bytesConf(ByteUnit.BYTE) - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) private[spark] val BARRIER_SYNC_TIMEOUT = ConfigBuilder("spark.barrier.sync.timeout") http://git-wip-us.apache.org/repos/asf/spark/blob/8fbc1830/core/src/main/scala/org/apache/spark/storage/DiskStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
spark git commit: [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager.
Repository: spark Updated Branches: refs/heads/master af3b81607 -> 68dde3481 [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager. This avoids having two classes to deal with tokens; now the above class is a one-stop shop for dealing with delegation tokens. The YARN backend extends that class instead of doing composition like before, resulting in a bit less code there too. The renewer functionality is basically the same code that used to be in YARN's AMCredentialRenewer. That is also the reason why the public API of HadoopDelegationTokenManager is a little bit odd; the YARN AM has some odd requirements for how this all should be initialized, and the weirdness is needed currently to support that. Tested: - YARN with stress app for DT renewal - Mesos and K8S with basic kerberos tests (both tgt and keytab) Closes #22624 from vanzin/SPARK-23781. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/68dde348 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/68dde348 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/68dde348 Branch: refs/heads/master Commit: 68dde3481ea458b0b8deeec2f99233c2d4c1e056 Parents: af3b816 Author: Marcelo Vanzin Authored: Wed Oct 31 13:00:10 2018 -0500 Committer: Imran Rashid Committed: Wed Oct 31 13:00:10 2018 -0500 -- .../main/scala/org/apache/spark/SparkConf.scala | 4 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 14 - .../security/HadoopDelegationTokenManager.scala | 278 ++- .../HadoopFSDelegationTokenProvider.scala | 5 +- .../apache/spark/internal/config/package.scala | 4 + .../cluster/CoarseGrainedSchedulerBackend.scala | 40 ++- .../HadoopDelegationTokenManagerSuite.scala | 142 +++--- .../spark/deploy/k8s/KubernetesConf.scala | 3 +- .../hadooputils/HadoopKerberosLogin.scala | 10 +- ...KubernetesHadoopDelegationTokenManager.scala | 35 +-- .../MesosCoarseGrainedSchedulerBackend.scala| 19 +- .../MesosHadoopDelegationTokenManager.scala | 160 --- .../spark/deploy/yarn/ApplicationMaster.scala | 24 +- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../org/apache/spark/deploy/yarn/config.scala | 4 - .../yarn/security/AMCredentialRenewer.scala | 177 .../YARNHadoopDelegationTokenManager.scala | 48 ++-- .../cluster/YarnSchedulerBackend.scala | 5 +- .../YARNHadoopDelegationTokenManagerSuite.scala | 5 +- 19 files changed, 355 insertions(+), 624 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/68dde348/core/src/main/scala/org/apache/spark/SparkConf.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 5166543..8537c53 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -731,7 +731,9 @@ private[spark] object SparkConf extends Logging { KEYTAB.key -> Seq( AlternateConfig("spark.yarn.keytab", "3.0")), PRINCIPAL.key -> Seq( - AlternateConfig("spark.yarn.principal", "3.0")) + AlternateConfig("spark.yarn.principal", "3.0")), +KERBEROS_RELOGIN_PERIOD.key -> Seq( + AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0")) ) /** http://git-wip-us.apache.org/repos/asf/spark/blob/68dde348/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 78a7cf6..5979151 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -414,20 +414,6 @@ object SparkHadoopUtil { def get: SparkHadoopUtil = instance /** - * Given an expiration date for the current set of credentials, calculate the time when new - * credentials should be created. - * - * @param expirationDate Drop-dead expiration date - * @param conf Spark configuration - * @return Timestamp when new credentials should be created. - */ - private[spark] def nextCredentialRenewalTime(expirationDate: Long, conf: SparkConf): Long = { -val ct = System.currentTimeMillis -val ratio = conf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) -(ct + (ratio * (expirationDate - ct))).toLong - } - - /** * Returns a Configuration object with Spark configuration applied on top. Unlike * the instance method, this will always return a Configuration instance, and
spark git commit: [SPARK-23429][CORE][FOLLOWUP] MetricGetter should rename to ExecutorMetricType in comments
Repository: spark Updated Branches: refs/heads/master ce40efa20 -> a129f0795 [SPARK-23429][CORE][FOLLOWUP] MetricGetter should rename to ExecutorMetricType in comments ## What changes were proposed in this pull request? MetricGetter should rename to ExecutorMetricType in comments. ## How was this patch tested? Just comments, no need to test. Closes #22884 from LantaoJin/SPARK-23429_FOLLOWUP. Authored-by: LantaoJin Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a129f079 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a129f079 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a129f079 Branch: refs/heads/master Commit: a129f079557204e3694754a5f9184c7f178cdf2a Parents: ce40efa Author: LantaoJin Authored: Tue Oct 30 11:01:55 2018 -0500 Committer: Imran Rashid Committed: Tue Oct 30 11:01:55 2018 -0500 -- core/src/main/scala/org/apache/spark/Heartbeater.scala | 2 +- .../src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/Heartbeater.scala -- diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 5ba1b9b..84091ee 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,7 +61,7 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by MetricGetter.values + * determined by ExecutorMetricType.values */ def getCurrentMetrics(): ExecutorMetrics = { val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala -- diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 2933f3b..1befd27 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -28,7 +28,7 @@ import org.apache.spark.metrics.ExecutorMetricType @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - // Metrics are indexed by MetricGetter.values + // Metrics are indexed by ExecutorMetricType.values private val metrics = new Array[Long](ExecutorMetricType.values.length) // the first element is initialized to -1, indicating that the values for the array http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f93d8a8..34b1160 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -265,7 +265,7 @@ private[spark] class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, - // executor metrics indexed by MetricGetter.values + // executor metrics indexed by ExecutorMetricType.values executorUpdates: ExecutorMetrics): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, Some(executorUpdates))) http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 293e836..e92b8a2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -175,7 +175,7 @@ case class SparkListenerExecutorMetricsUpdate( * @param execId executor id * @param stageId stage id * @param stageAttemptId stage attempt - * @param executorMetrics executor level metrics,
spark git commit: [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue
Repository: spark Updated Branches: refs/heads/branch-2.4 9c0c6d4d5 -> 1001d2314 [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave a little extra room. This is necessary when reading blocks >2GB off the network (for remote reads or for cache replication). Unit tests via jenkins, ran a test with blocks over 2gb on a cluster Closes #22705 from squito/SPARK-25704. Authored-by: Imran Rashid Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1001d231 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1001d231 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1001d231 Branch: refs/heads/branch-2.4 Commit: 1001d2314275c902da519725da266a23b537e33a Parents: 9c0c6d4 Author: Imran Rashid Authored: Fri Oct 19 12:52:41 2018 -0500 Committer: Imran Rashid Committed: Fri Oct 19 12:54:08 2018 -0500 -- .../org/apache/spark/storage/BlockManager.scala | 6 ++ .../apache/spark/util/io/ChunkedByteBuffer.scala| 16 +--- 2 files changed, 11 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1001d231/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0fe82ac..c01a453 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -133,8 +133,6 @@ private[spark] class BlockManager( private[spark] val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED) - private val chunkSize = -conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt private val remoteReadNioBufferConversion = conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) @@ -451,7 +449,7 @@ private[spark] class BlockManager( new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) case None => -ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) +ChunkedByteBuffer.fromFile(tmpFile) } putBytes(blockId, buffer, level)(classTag) tmpFile.delete() @@ -797,7 +795,7 @@ private[spark] class BlockManager( if (remoteReadNioBufferConversion) { return Some(new ChunkedByteBuffer(data.nioByteBuffer())) } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) + return Some(ChunkedByteBuffer.fromManagedBuffer(data)) } } logDebug(s"The value of block $blockId is null") http://git-wip-us.apache.org/repos/asf/spark/blob/1001d231/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 4aa8d45..9547cb4 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} import org.apache.spark.storage.StorageUtils +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils /** @@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { } -object ChunkedByteBuffer { +private[spark] object ChunkedByteBuffer { + + // TODO eliminate this method if we switch BlockManager to getting InputStreams - def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { + def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => -fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) +fromFile(f.getFile, f.getOffset, f.getLength) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { -fromFile(file, maxChunkSize, 0, file.length()) + def fromFile(file: File): ChunkedByteBuffer = { +fromFile(file, 0, file.length()) } private def fromFile( file: File, - maxChunkSize: Int, offset: Long, length: Long):
spark git commit: [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue
Repository: spark Updated Branches: refs/heads/master 130121711 -> 43717dee5 [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave a little extra room. This is necessary when reading blocks >2GB off the network (for remote reads or for cache replication). Unit tests via jenkins, ran a test with blocks over 2gb on a cluster Closes #22705 from squito/SPARK-25704. Authored-by: Imran Rashid Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/43717dee Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/43717dee Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/43717dee Branch: refs/heads/master Commit: 43717dee570dc41d71f0b27b8939f6297a029a02 Parents: 1301217 Author: Imran Rashid Authored: Fri Oct 19 12:52:41 2018 -0500 Committer: Imran Rashid Committed: Fri Oct 19 12:52:41 2018 -0500 -- .../org/apache/spark/storage/BlockManager.scala | 6 ++ .../apache/spark/util/io/ChunkedByteBuffer.scala| 16 +--- 2 files changed, 11 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/43717dee/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0fe82ac..c01a453 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -133,8 +133,6 @@ private[spark] class BlockManager( private[spark] val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED) - private val chunkSize = -conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt private val remoteReadNioBufferConversion = conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) @@ -451,7 +449,7 @@ private[spark] class BlockManager( new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) case None => -ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) +ChunkedByteBuffer.fromFile(tmpFile) } putBytes(blockId, buffer, level)(classTag) tmpFile.delete() @@ -797,7 +795,7 @@ private[spark] class BlockManager( if (remoteReadNioBufferConversion) { return Some(new ChunkedByteBuffer(data.nioByteBuffer())) } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) + return Some(ChunkedByteBuffer.fromManagedBuffer(data)) } } logDebug(s"The value of block $blockId is null") http://git-wip-us.apache.org/repos/asf/spark/blob/43717dee/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 4aa8d45..9547cb4 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} import org.apache.spark.storage.StorageUtils +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils /** @@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { } -object ChunkedByteBuffer { +private[spark] object ChunkedByteBuffer { + + // TODO eliminate this method if we switch BlockManager to getting InputStreams - def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { + def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => -fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) +fromFile(f.getFile, f.getOffset, f.getLength) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { -fromFile(file, maxChunkSize, 0, file.length()) + def fromFile(file: File): ChunkedByteBuffer = { +fromFile(file, 0, file.length()) } private def fromFile( file: File, - maxChunkSize: Int, offset: Long, length: Long):
spark git commit: [SPARK-20327][YARN] Follow up: fix resource request tests on Hadoop 3.
Repository: spark Updated Branches: refs/heads/master 24f5bbd77 -> 7d425b190 [SPARK-20327][YARN] Follow up: fix resource request tests on Hadoop 3. The test fix is to allocate a `Resource` object only after the resource types have been initialized. Otherwise the YARN classes get in a weird state and throw a different exception than expected, because the resource has a different view of the registered resources. I also removed a test for a null resource since that seems unnecessary and made the fix more complicated. All the other changes are just cleanup; basically simplify the tests by defining what is being tested and deriving the resource type registration and the SparkConf from that data, instead of having redundant definitions in the tests. Ran tests with Hadoop 3 (and also without it). Closes #22751 from vanzin/SPARK-20327.fix. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7d425b19 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7d425b19 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7d425b19 Branch: refs/heads/master Commit: 7d425b190a91f49193eeb58c398e497ff92c6169 Parents: 24f5bbd Author: Marcelo Vanzin Authored: Wed Oct 17 10:40:47 2018 -0500 Committer: Imran Rashid Committed: Wed Oct 17 10:40:47 2018 -0500 -- .../deploy/yarn/ResourceRequestHelper.scala | 4 +- .../apache/spark/deploy/yarn/ClientSuite.scala | 70 +++ .../yarn/ResourceRequestHelperSuite.scala | 181 ++- .../deploy/yarn/ResourceRequestTestHelper.scala | 13 +- 4 files changed, 90 insertions(+), 178 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7d425b19/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala -- diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index 9534f3a..012268e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{CausedBy, Utils} /** * This helper class uses some of Hadoop 3 methods from the YARN API, @@ -121,6 +121,8 @@ private object ResourceRequestHelper extends Logging { case _: MatchError => throw new IllegalArgumentException(s"Resource request for '$name' ('$rawAmount') " + s"does not match pattern $AMOUNT_AND_UNIT_REGEX.") +case CausedBy(e: IllegalArgumentException) => + throw new IllegalArgumentException(s"Invalid request for $name: ${e.getMessage}") case e: InvocationTargetException if e.getCause != null => throw e.getCause } } http://git-wip-us.apache.org/repos/asf/spark/blob/7d425b19/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala -- diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 533cb2b..b3286e8 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -200,20 +200,6 @@ class ClientSuite extends SparkFunSuite with Matchers { appContext.getMaxAppAttempts should be (42) } - test("resource request (client mode)") { -val sparkConf = new SparkConf().set("spark.submit.deployMode", "client") - .set(YARN_AM_RESOURCE_TYPES_PREFIX + "fpga", "2") - .set(YARN_AM_RESOURCE_TYPES_PREFIX + "gpu", "3") -testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 2), ("gpu", 3))) - } - - test("resource request (cluster mode)") { -val sparkConf = new SparkConf().set("spark.submit.deployMode", "cluster") - .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "fpga", "4") - .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "gpu", "5") -testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 4), ("gpu", 5))) - } - test("spark.yarn.jars with multiple paths and globs") { val libs = Utils.createTempDir() val single = Utils.createTempDir() @@ -372,6 +358,35 @@ class ClientSuite
spark git commit: [SPARK-25535][CORE] Work around bad error handling in commons-crypto.
Repository: spark Updated Branches: refs/heads/master deb9588b2 -> 3eee9e024 [SPARK-25535][CORE] Work around bad error handling in commons-crypto. The commons-crypto library does some questionable error handling internally, which can lead to JVM crashes if some call into native code fails and cleans up state it should not. While the library is not fixed, this change adds some workarounds in Spark code so that when an error is detected in the commons-crypto side, Spark avoids calling into the library further. Tested with existing and added unit tests. Closes #22557 from vanzin/SPARK-25535. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3eee9e02 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3eee9e02 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3eee9e02 Branch: refs/heads/master Commit: 3eee9e02463e10570a29fad00823c953debd945e Parents: deb9588 Author: Marcelo Vanzin Authored: Tue Oct 9 09:27:08 2018 -0500 Committer: Imran Rashid Committed: Tue Oct 9 09:27:08 2018 -0500 -- .../apache/spark/network/crypto/AuthEngine.java | 95 - .../spark/network/crypto/TransportCipher.java | 60 ++-- .../spark/network/crypto/AuthEngineSuite.java | 17 +++ .../spark/security/CryptoStreamUtils.scala | 137 +-- .../spark/security/CryptoStreamUtilsSuite.scala | 37 - 5 files changed, 295 insertions(+), 51 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3eee9e02/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java -- diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java index 056505e..64fdb32 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -159,15 +159,21 @@ class AuthEngine implements Closeable { // accurately report the errors when they happen. RuntimeException error = null; byte[] dummy = new byte[8]; -try { - doCipherOp(encryptor, dummy, true); -} catch (Exception e) { - error = new RuntimeException(e); +if (encryptor != null) { + try { +doCipherOp(Cipher.ENCRYPT_MODE, dummy, true); + } catch (Exception e) { +error = new RuntimeException(e); + } + encryptor = null; } -try { - doCipherOp(decryptor, dummy, true); -} catch (Exception e) { - error = new RuntimeException(e); +if (decryptor != null) { + try { +doCipherOp(Cipher.DECRYPT_MODE, dummy, true); + } catch (Exception e) { +error = new RuntimeException(e); + } + decryptor = null; } random.close(); @@ -189,11 +195,11 @@ class AuthEngine implements Closeable { } private byte[] decrypt(byte[] in) throws GeneralSecurityException { -return doCipherOp(decryptor, in, false); +return doCipherOp(Cipher.DECRYPT_MODE, in, false); } private byte[] encrypt(byte[] in) throws GeneralSecurityException { -return doCipherOp(encryptor, in, false); +return doCipherOp(Cipher.ENCRYPT_MODE, in, false); } private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) @@ -205,11 +211,13 @@ class AuthEngine implements Closeable { byte[] iv = new byte[conf.ivLength()]; System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); -encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); -encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); +CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); +_encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); +this.encryptor = _encryptor; -decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); -decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); +CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); +_decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); +this.decryptor = _decryptor; } /** @@ -241,29 +249,52 @@ class AuthEngine implements Closeable { return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); } - private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal) + private byte[] doCipherOp(int mode, byte[] in, boolean isFinal) throws GeneralSecurityException { -Preconditions.checkState(cipher != null); +CryptoCipher cipher; +switch (mode) { + case
[3/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue
[PYSPARK][SQL] Updates to RowQueue Tested with updates to RowQueueSuite (cherry picked from commit 6d742d1bd71aa3803dce91a830b37284cb18cf70) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4f10aff4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4f10aff4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4f10aff4 Branch: refs/heads/branch-2.2 Commit: 4f10aff403ccc8287a816cb94ddf7f11e185907a Parents: dd0e7cf Author: Imran Rashid Authored: Thu Sep 6 12:11:47 2018 -0500 Committer: Imran Rashid Committed: Tue Sep 25 11:46:06 2018 -0500 -- .../spark/sql/execution/python/RowQueue.scala | 27 ++- .../sql/execution/python/RowQueueSuite.scala| 28 +++- 2 files changed, 41 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4f10aff4/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index cd1e77f..4d6820c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -21,9 +21,10 @@ import java.io._ import com.google.common.io.Closeables -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.memory.MemoryBlock @@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any * reader has begun reading from the queue. */ -private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { - private var out = new DataOutputStream( -new BufferedOutputStream(new FileOutputStream(file.toString))) +private[python] case class DiskRowQueue( +file: File, +fields: Int, +serMgr: SerializerManager) extends RowQueue { + + private var out = new DataOutputStream(serMgr.wrapForEncryption( +new BufferedOutputStream(new FileOutputStream(file.toString private var unreadBytes = 0L private var in: DataInputStream = _ @@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new NioBufferedFileInputStream(file)) + in = new DataInputStream(serMgr.wrapForEncryption( +new NioBufferedFileInputStream(file))) } if (unreadBytes > 0) { @@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu private[python] case class HybridRowQueue( memManager: TaskMemoryManager, tempDir: File, -numFields: Int) +numFields: Int, +serMgr: SerializerManager) extends MemoryConsumer(memManager) with RowQueue { // Each buffer should have at least one row @@ -212,7 +219,7 @@ private[python] case class HybridRowQueue( } private def createDiskQueue(): RowQueue = { -DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) +DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr) } private def createNewQueue(required: Long): RowQueue = { @@ -279,3 +286,9 @@ private[python] case class HybridRowQueue( } } } + +private[python] object HybridRowQueue { + def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): HybridRowQueue = { +HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager) + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/4f10aff4/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33c..1ec9986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} -import
[1/3] spark git commit: [SPARK-25253][PYSPARK] Refactor local connection & auth code
Repository: spark Updated Branches: refs/heads/branch-2.2 bd12eb75d -> 4f10aff40 [SPARK-25253][PYSPARK] Refactor local connection & auth code This eliminates some duplication in the code to connect to a server on localhost to talk directly to the jvm. Also it gives consistent ipv6 and error handling. Two other incidental changes, that shouldn't matter: 1) python barrier tasks perform authentication immediately (rather than waiting for the BARRIER_FUNCTION indicator) 2) for `rdd._load_from_socket`, the timeout is only increased after authentication. Closes #22247 from squito/py_connection_refactor. Authored-by: Imran Rashid Signed-off-by: hyukjinkwon (cherry picked from commit 38391c9aa8a88fcebb337934f30298a32d91596b) (cherry picked from commit a2a54a5f49364a1825932c9f04eb0ff82dd7d465) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fc1c4e7d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fc1c4e7d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fc1c4e7d Branch: refs/heads/branch-2.2 Commit: fc1c4e7d24f7d0afb3b79d66aa9812e7dddc2f38 Parents: bd12eb7 Author: Imran Rashid Authored: Wed Aug 29 09:47:38 2018 +0800 Committer: Imran Rashid Committed: Tue Sep 25 11:45:59 2018 -0500 -- python/pyspark/java_gateway.py | 32 +++- python/pyspark/rdd.py | 24 ++-- python/pyspark/worker.py | 7 ++- 3 files changed, 35 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fc1c4e7d/python/pyspark/java_gateway.py -- diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 7abf2c1..191dfce 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -133,7 +133,7 @@ def launch_gateway(conf=None): return gateway -def do_server_auth(conn, auth_secret): +def _do_server_auth(conn, auth_secret): """ Performs the authentication protocol defined by the SocketAuthHelper class on the given file-like object 'conn'. @@ -144,3 +144,33 @@ def do_server_auth(conn, auth_secret): if reply != "ok": conn.close() raise Exception("Unexpected reply from iterator server.") + + +def local_connect_and_auth(port, auth_secret): +""" +Connect to local host, authenticate with it, and return a (sockfile,sock) for that connection. +Handles IPV4 & IPV6, does some error handling. +:param port +:param auth_secret +:return: a tuple with (sockfile, sock) +""" +sock = None +errors = [] +# Support for both IPv4 and IPv6. +# On most of IPv6-ready systems, IPv6 will take precedence. +for res in socket.getaddrinfo("127.0.0.1", port, socket.AF_UNSPEC, socket.SOCK_STREAM): +af, socktype, proto, _, sa = res +try: +sock = socket.socket(af, socktype, proto) +sock.settimeout(15) +sock.connect(sa) +sockfile = sock.makefile("rwb", 65536) +_do_server_auth(sockfile, auth_secret) +return (sockfile, sock) +except socket.error as e: +emsg = _exception_message(e) +errors.append("tried to connect to %s, but an error occured: %s" % (sa, emsg)) +sock.close() +sock = None +else: +raise Exception("could not open socket: %s" % errors) http://git-wip-us.apache.org/repos/asf/spark/blob/fc1c4e7d/python/pyspark/rdd.py -- diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 864cebb..7d84cbd 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -39,7 +39,7 @@ if sys.version > '3': else: from itertools import imap as map, ifilter as filter -from pyspark.java_gateway import do_server_auth +from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \ @@ -122,30 +122,10 @@ def _parse_memory(s): def _load_from_socket(sock_info, serializer): -port, auth_secret = sock_info -sock = None -# Support for both IPv4 and IPv6. -# On most of IPv6-ready systems, IPv6 will take precedence. -for res in socket.getaddrinfo("localhost", port, socket.AF_UNSPEC, socket.SOCK_STREAM): -af, socktype, proto, canonname, sa = res -sock = socket.socket(af, socktype, proto) -try: -sock.settimeout(15) -sock.connect(sa) -except socket.error: -sock.close() -sock = None -continue -break -
[2/3] spark git commit: [PYSPARK] Updates to pyspark broadcast
[PYSPARK] Updates to pyspark broadcast (cherry picked from commit 09dd34cb1706f2477a89174d6a1a0f17ed5b0a65) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dd0e7cf5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dd0e7cf5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dd0e7cf5 Branch: refs/heads/branch-2.2 Commit: dd0e7cf5287148618404593ca095dd900b6e993f Parents: fc1c4e7 Author: Imran Rashid Authored: Mon Aug 13 21:35:34 2018 -0500 Committer: Imran Rashid Committed: Tue Sep 25 11:46:03 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 349 --- .../spark/api/python/PythonRDDSuite.scala | 23 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/broadcast.py | 58 ++- python/pyspark/context.py | 63 +++- python/pyspark/serializers.py | 58 +++ python/pyspark/test_broadcast.py| 126 +++ python/pyspark/test_serializers.py | 90 + python/pyspark/worker.py| 24 +- 9 files changed, 705 insertions(+), 88 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dd0e7cf5/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 7b5a179..2f4e3bc 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.Promise +import scala.concurrent.duration.Duration import scala.language.existentials -import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -293,19 +296,51 @@ private[spark] class PythonRunner( val newBids = broadcastVars.map(_.id).toSet // number of different broadcasts val toRemove = oldBids.diff(newBids) -val cnt = toRemove.size + newBids.diff(oldBids).size +val addedBids = newBids.diff(oldBids) +val cnt = toRemove.size + addedBids.size +val needsDecryptionServer = env.serializerManager.encryptionEnabled && addedBids.nonEmpty +dataOut.writeBoolean(needsDecryptionServer) dataOut.writeInt(cnt) -for (bid <- toRemove) { - // remove the broadcast from worker - dataOut.writeLong(- bid - 1) // bid >= 0 - oldBids.remove(bid) +def sendBidsToRemove(): Unit = { + for (bid <- toRemove) { +// remove the broadcast from worker +dataOut.writeLong(-bid - 1) // bid >= 0 +oldBids.remove(bid) + } } -for (broadcast <- broadcastVars) { - if (!oldBids.contains(broadcast.id)) { +if (needsDecryptionServer) { + // if there is encryption, we setup a server which reads the encrypted files, and sends +// the decrypted data to python + val idsAndFiles = broadcastVars.flatMap { broadcast => + if (oldBids.contains(broadcast.id)) { + None +} else { + Some((broadcast.id, broadcast.value.path)) +} +} + val server = new EncryptedPythonBroadcastServer(env, idsAndFiles) + dataOut.writeInt(server.port) + logTrace(s"broadcast decryption server setup on ${server.port}") + PythonRDD.writeUTF(server.secret, dataOut) + sendBidsToRemove() + idsAndFiles.foreach { case (id, _) => // send new broadcast -dataOut.writeLong(broadcast.id) -PythonRDD.writeUTF(broadcast.value.path, dataOut) -oldBids.add(broadcast.id) +dataOut.writeLong(id) +oldBids.add(id) + } + dataOut.flush() + logTrace("waiting for python to read decrypted broadcast data from server") + server.waitTillBroadcastDataSent() + logTrace("done sending decrypted data to python") +} else { +
spark git commit: [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite
Repository: spark Updated Branches: refs/heads/master 123f0041d -> a6f37b074 [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite PythonForeachWriterSuite was failing because RowQueue now needs to have a handle on a SparkEnv with a SerializerManager, so added a mock env with a serializer manager. Also fixed a typo in the `finally` that was hiding the real exception. Tested PythonForeachWriterSuite locally, full tests via jenkins. Closes #22452 from squito/SPARK-25456. Authored-by: Imran Rashid Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a6f37b07 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a6f37b07 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a6f37b07 Branch: refs/heads/master Commit: a6f37b0742d87d5c8ee3e134999d665e5719e822 Parents: 123f004 Author: Imran Rashid Authored: Tue Sep 18 16:33:37 2018 -0500 Committer: Imran Rashid Committed: Tue Sep 18 16:33:37 2018 -0500 -- .../execution/python/PythonForeachWriterSuite.scala | 14 +++--- 1 file changed, 11 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a6f37b07/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala index 07e6034..d02014c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala @@ -19,17 +19,20 @@ package org.apache.spark.sql.execution.python import scala.collection.mutable.ArrayBuffer +import org.mockito.Mockito.when import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.python.PythonForeachWriter.UnsafeRowBuffer import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.util.Utils -class PythonForeachWriterSuite extends SparkFunSuite with Eventually { +class PythonForeachWriterSuite extends SparkFunSuite with Eventually with MockitoSugar { testWithBuffer("UnsafeRowBuffer: iterator blocks when no data is available") { b => b.assertIteratorBlocked() @@ -75,7 +78,7 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { tester = new BufferTester(memBytes, sleepPerRowReadMs) f(tester) } finally { -if (tester == null) tester.close() +if (tester != null) tester.close() } } } @@ -83,7 +86,12 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { class BufferTester(memBytes: Long, sleepPerRowReadMs: Int) { private val buffer = { - val mem = new TestMemoryManager(new SparkConf()) + val mockEnv = mock[SparkEnv] + val conf = new SparkConf() + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf, None) + when(mockEnv.serializerManager).thenReturn(serializerManager) + SparkEnv.set(mockEnv) + val mem = new TestMemoryManager(conf) mem.limit(memBytes) val taskM = new TaskMemoryManager(mem, 0) new UnsafeRowBuffer(taskM, Utils.createTempDir(), 1) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite
Repository: spark Updated Branches: refs/heads/branch-2.4 67f2cb6e0 -> 76514a015 [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite PythonForeachWriterSuite was failing because RowQueue now needs to have a handle on a SparkEnv with a SerializerManager, so added a mock env with a serializer manager. Also fixed a typo in the `finally` that was hiding the real exception. Tested PythonForeachWriterSuite locally, full tests via jenkins. Closes #22452 from squito/SPARK-25456. Authored-by: Imran Rashid Signed-off-by: Imran Rashid (cherry picked from commit a6f37b0742d87d5c8ee3e134999d665e5719e822) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/76514a01 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/76514a01 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/76514a01 Branch: refs/heads/branch-2.4 Commit: 76514a015168de8d8b54b3abf6b835050eefd8c2 Parents: 67f2cb6 Author: Imran Rashid Authored: Tue Sep 18 16:33:37 2018 -0500 Committer: Imran Rashid Committed: Tue Sep 18 16:33:49 2018 -0500 -- .../execution/python/PythonForeachWriterSuite.scala | 14 +++--- 1 file changed, 11 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/76514a01/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala index 07e6034..d02014c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala @@ -19,17 +19,20 @@ package org.apache.spark.sql.execution.python import scala.collection.mutable.ArrayBuffer +import org.mockito.Mockito.when import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.python.PythonForeachWriter.UnsafeRowBuffer import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.util.Utils -class PythonForeachWriterSuite extends SparkFunSuite with Eventually { +class PythonForeachWriterSuite extends SparkFunSuite with Eventually with MockitoSugar { testWithBuffer("UnsafeRowBuffer: iterator blocks when no data is available") { b => b.assertIteratorBlocked() @@ -75,7 +78,7 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { tester = new BufferTester(memBytes, sleepPerRowReadMs) f(tester) } finally { -if (tester == null) tester.close() +if (tester != null) tester.close() } } } @@ -83,7 +86,12 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { class BufferTester(memBytes: Long, sleepPerRowReadMs: Int) { private val buffer = { - val mem = new TestMemoryManager(new SparkConf()) + val mockEnv = mock[SparkEnv] + val conf = new SparkConf() + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf, None) + when(mockEnv.serializerManager).thenReturn(serializerManager) + SparkEnv.set(mockEnv) + val mem = new TestMemoryManager(conf) mem.limit(memBytes) val taskM = new TaskMemoryManager(mem, 0) new UnsafeRowBuffer(taskM, Utils.createTempDir(), 1) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[1/3] spark git commit: [PYSPARK] Updates to pyspark broadcast
Repository: spark Updated Branches: refs/heads/branch-2.4 08f7b145c -> 7beb34175 [PYSPARK] Updates to pyspark broadcast (cherry picked from commit 58419b92673c46911c25bc6c6b13397f880c6424) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/963af130 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/963af130 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/963af130 Branch: refs/heads/branch-2.4 Commit: 963af1301131f63a955288edcd6bc76956bb6447 Parents: 08f7b14 Author: Imran Rashid Authored: Mon Aug 13 21:35:34 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 15:28:31 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 299 --- .../apache/spark/api/python/PythonRunner.scala | 52 +++- .../spark/api/python/PythonRDDSuite.scala | 23 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/broadcast.py | 58 +++- python/pyspark/context.py | 64 ++-- python/pyspark/serializers.py | 51 python/pyspark/sql/session.py | 12 +- python/pyspark/sql/tests.py | 45 ++- python/pyspark/test_broadcast.py| 126 python/pyspark/test_serializers.py | 90 ++ python/pyspark/tests.py | 9 +- python/pyspark/worker.py| 22 +- .../spark/sql/api/python/PythonSQLUtils.scala | 47 ++- .../sql/execution/arrow/ArrowConverters.scala | 9 +- 15 files changed, 789 insertions(+), 120 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/963af130/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e639a84..8b5a7a9 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.Promise +import scala.concurrent.duration.Duration import scala.language.existentials -import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -169,27 +172,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { -val file = new DataInputStream(new FileInputStream(filename)) +readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + } + + def readRDDFromInputStream( + sc: SparkContext, + in: InputStream, + parallelism: Int): JavaRDD[Array[Byte]] = { +val din = new DataInputStream(in) try { val objs = new mutable.ArrayBuffer[Array[Byte]] try { while (true) { - val length = file.readInt() + val length = din.readInt() val obj = new Array[Byte](length) - file.readFully(obj) + din.readFully(obj) objs += obj } } catch { case eof: EOFException => // No-op } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) } finally { - file.close() + din.close() } } - def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { -sc.broadcast(new PythonBroadcast(path)) + def setupBroadcast(path: String): PythonBroadcast = { +new PythonBroadcast(path) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -419,34 +429,15 @@ private[spark] object PythonRDD extends Logging { */ private[spark] def serveToStream( threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = { -val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) -// Close the socket if no connection in 15 seconds -serverSocket.setSoTimeout(15000) - -new Thread(threadName) { - setDaemon(true) - override def run() { -
[3/3] spark git commit: [CORE] Updates to remote cache reads
[CORE] Updates to remote cache reads Covered by tests in DistributedSuite (cherry picked from commit a97001d21757ae214c86371141bd78a376200f66) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7beb3417 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7beb3417 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7beb3417 Branch: refs/heads/branch-2.4 Commit: 7beb3417591b0f9c436d9175f3343ee79795d536 Parents: 80e317b Author: Imran Rashid Authored: Wed Aug 22 16:38:28 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 15:28:40 2018 -0500 -- .../spark/network/buffer/ManagedBuffer.java | 5 +- .../spark/network/shuffle/DownloadFile.java | 47 ++ .../network/shuffle/DownloadFileManager.java| 36 .../shuffle/DownloadFileWritableChannel.java| 30 +++ .../network/shuffle/ExternalShuffleClient.java | 4 +- .../network/shuffle/OneForOneBlockFetcher.java | 28 +++--- .../spark/network/shuffle/ShuffleClient.java| 4 +- .../network/shuffle/SimpleDownloadFile.java | 91 .../spark/network/shuffle/TempFileManager.java | 36 .../spark/network/BlockTransferService.scala| 6 +- .../netty/NettyBlockTransferService.scala | 4 +- .../org/apache/spark/storage/BlockManager.scala | 78 ++--- .../org/apache/spark/storage/DiskStore.scala| 16 .../storage/ShuffleBlockFetcherIterator.scala | 21 +++-- .../spark/storage/BlockManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 6 +- 16 files changed, 328 insertions(+), 92 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7beb3417/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java -- diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index 1861f8d..2d573f5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -36,7 +36,10 @@ import java.nio.ByteBuffer; */ public abstract class ManagedBuffer { - /** Number of bytes of the data. */ + /** + * Number of bytes of the data. If this buffer will decrypt for all of the views into the data, + * this is the size of the decrypted data. + */ public abstract long size(); /** http://git-wip-us.apache.org/repos/asf/spark/blob/7beb3417/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java -- diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java new file mode 100644 index 000..633622b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; + +/** + * A handle on the file used when fetching remote data to disk. Used to ensure the lifecycle of + * writing the data, reading it back, and then cleaning it up is followed. Specific implementations + * may also handle encryption. The data can be read only via DownloadFileWritableChannel, + * which ensures data is not read until after the writer is closed. + */ +public interface DownloadFile { + /** + * Delete the file. + * + * @return true if and only if the file or directory is + * successfully deleted; false otherwise + */ + boolean delete(); + + /** + * A channel for writing data to the file. This special channel allows access to the data for + * reading, after the channel is closed, via {@link
[2/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue
[PYSPARK][SQL] Updates to RowQueue Tested with updates to RowQueueSuite (cherry picked from commit 8f5a5a9e5b9f273443b2721f80c99dc7397ef4c0) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/80e317b3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/80e317b3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/80e317b3 Branch: refs/heads/branch-2.4 Commit: 80e317b3a5cafb95a5b5a00d1d8b3b8c595985e2 Parents: 963af13 Author: Imran Rashid Authored: Thu Sep 6 12:11:47 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 15:28:36 2018 -0500 -- .../spark/sql/execution/python/RowQueue.scala | 27 ++- .../sql/execution/python/RowQueueSuite.scala| 28 +++- 2 files changed, 41 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/80e317b3/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index e2fa6e7..d2820ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -21,9 +21,10 @@ import java.io._ import com.google.common.io.Closeables -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.memory.MemoryBlock @@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any * reader has begun reading from the queue. */ -private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { - private var out = new DataOutputStream( -new BufferedOutputStream(new FileOutputStream(file.toString))) +private[python] case class DiskRowQueue( +file: File, +fields: Int, +serMgr: SerializerManager) extends RowQueue { + + private var out = new DataOutputStream(serMgr.wrapForEncryption( +new BufferedOutputStream(new FileOutputStream(file.toString private var unreadBytes = 0L private var in: DataInputStream = _ @@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new NioBufferedFileInputStream(file)) + in = new DataInputStream(serMgr.wrapForEncryption( +new NioBufferedFileInputStream(file))) } if (unreadBytes > 0) { @@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu private[python] case class HybridRowQueue( memManager: TaskMemoryManager, tempDir: File, -numFields: Int) +numFields: Int, +serMgr: SerializerManager) extends MemoryConsumer(memManager) with RowQueue { // Each buffer should have at least one row @@ -212,7 +219,7 @@ private[python] case class HybridRowQueue( } private def createDiskQueue(): RowQueue = { -DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) +DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr) } private def createNewQueue(required: Long): RowQueue = { @@ -279,3 +286,9 @@ private[python] case class HybridRowQueue( } } } + +private[python] object HybridRowQueue { + def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): HybridRowQueue = { +HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager) + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/80e317b3/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33c..1ec9986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} -import
[1/3] spark git commit: [CORE] Updates to remote cache reads
Repository: spark Updated Branches: refs/heads/master 553af22f2 -> a97001d21 [CORE] Updates to remote cache reads Covered by tests in DistributedSuite Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a97001d2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a97001d2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a97001d2 Branch: refs/heads/master Commit: a97001d21757ae214c86371141bd78a376200f66 Parents: 8f5a5a9 Author: Imran Rashid Authored: Wed Aug 22 16:38:28 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 14:06:09 2018 -0500 -- .../spark/network/buffer/ManagedBuffer.java | 5 +- .../spark/network/shuffle/DownloadFile.java | 47 ++ .../network/shuffle/DownloadFileManager.java| 36 .../shuffle/DownloadFileWritableChannel.java| 30 +++ .../network/shuffle/ExternalShuffleClient.java | 4 +- .../network/shuffle/OneForOneBlockFetcher.java | 28 +++--- .../spark/network/shuffle/ShuffleClient.java| 4 +- .../network/shuffle/SimpleDownloadFile.java | 91 .../spark/network/shuffle/TempFileManager.java | 36 .../spark/network/BlockTransferService.scala| 6 +- .../netty/NettyBlockTransferService.scala | 4 +- .../org/apache/spark/storage/BlockManager.scala | 78 ++--- .../org/apache/spark/storage/DiskStore.scala| 16 .../storage/ShuffleBlockFetcherIterator.scala | 21 +++-- .../spark/storage/BlockManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 6 +- 16 files changed, 328 insertions(+), 92 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a97001d2/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java -- diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index 1861f8d..2d573f5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -36,7 +36,10 @@ import java.nio.ByteBuffer; */ public abstract class ManagedBuffer { - /** Number of bytes of the data. */ + /** + * Number of bytes of the data. If this buffer will decrypt for all of the views into the data, + * this is the size of the decrypted data. + */ public abstract long size(); /** http://git-wip-us.apache.org/repos/asf/spark/blob/a97001d2/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java -- diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java new file mode 100644 index 000..633622b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; + +/** + * A handle on the file used when fetching remote data to disk. Used to ensure the lifecycle of + * writing the data, reading it back, and then cleaning it up is followed. Specific implementations + * may also handle encryption. The data can be read only via DownloadFileWritableChannel, + * which ensures data is not read until after the writer is closed. + */ +public interface DownloadFile { + /** + * Delete the file. + * + * @return true if and only if the file or directory is + * successfully deleted; false otherwise + */ + boolean delete(); + + /** + * A channel for writing data to the file. This special channel allows access to the data for + * reading, after the channel is closed, via
[3/3] spark git commit: [PYSPARK] Updates to pyspark broadcast
[PYSPARK] Updates to pyspark broadcast Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/58419b92 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/58419b92 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/58419b92 Branch: refs/heads/master Commit: 58419b92673c46911c25bc6c6b13397f880c6424 Parents: 553af22 Author: Imran Rashid Authored: Mon Aug 13 21:35:34 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 14:06:09 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 299 --- .../apache/spark/api/python/PythonRunner.scala | 52 +++- .../spark/api/python/PythonRDDSuite.scala | 23 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/broadcast.py | 58 +++- python/pyspark/context.py | 64 ++-- python/pyspark/serializers.py | 51 python/pyspark/sql/session.py | 12 +- python/pyspark/sql/tests.py | 45 ++- python/pyspark/test_broadcast.py| 126 python/pyspark/test_serializers.py | 90 ++ python/pyspark/tests.py | 9 +- python/pyspark/worker.py| 22 +- .../spark/sql/api/python/PythonSQLUtils.scala | 47 ++- .../sql/execution/arrow/ArrowConverters.scala | 9 +- 15 files changed, 789 insertions(+), 120 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/58419b92/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e639a84..8b5a7a9 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.Promise +import scala.concurrent.duration.Duration import scala.language.existentials -import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -169,27 +172,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { -val file = new DataInputStream(new FileInputStream(filename)) +readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + } + + def readRDDFromInputStream( + sc: SparkContext, + in: InputStream, + parallelism: Int): JavaRDD[Array[Byte]] = { +val din = new DataInputStream(in) try { val objs = new mutable.ArrayBuffer[Array[Byte]] try { while (true) { - val length = file.readInt() + val length = din.readInt() val obj = new Array[Byte](length) - file.readFully(obj) + din.readFully(obj) objs += obj } } catch { case eof: EOFException => // No-op } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) } finally { - file.close() + din.close() } } - def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { -sc.broadcast(new PythonBroadcast(path)) + def setupBroadcast(path: String): PythonBroadcast = { +new PythonBroadcast(path) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -419,34 +429,15 @@ private[spark] object PythonRDD extends Logging { */ private[spark] def serveToStream( threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = { -val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) -// Close the socket if no connection in 15 seconds -serverSocket.setSoTimeout(15000) - -new Thread(threadName) { - setDaemon(true) - override def run() { -try { - val sock = serverSocket.accept() - authHelper.authClient(sock) - - val out = new BufferedOutputStream(sock.getOutputStream) -
[2/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue
[PYSPARK][SQL] Updates to RowQueue Tested with updates to RowQueueSuite Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8f5a5a9e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8f5a5a9e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8f5a5a9e Branch: refs/heads/master Commit: 8f5a5a9e5b9f273443b2721f80c99dc7397ef4c0 Parents: 58419b9 Author: Imran Rashid Authored: Thu Sep 6 12:11:47 2018 -0500 Committer: Imran Rashid Committed: Mon Sep 17 14:06:09 2018 -0500 -- .../spark/sql/execution/python/RowQueue.scala | 27 ++- .../sql/execution/python/RowQueueSuite.scala| 28 +++- 2 files changed, 41 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8f5a5a9e/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index e2fa6e7..d2820ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -21,9 +21,10 @@ import java.io._ import com.google.common.io.Closeables -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.memory.MemoryBlock @@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any * reader has begun reading from the queue. */ -private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { - private var out = new DataOutputStream( -new BufferedOutputStream(new FileOutputStream(file.toString))) +private[python] case class DiskRowQueue( +file: File, +fields: Int, +serMgr: SerializerManager) extends RowQueue { + + private var out = new DataOutputStream(serMgr.wrapForEncryption( +new BufferedOutputStream(new FileOutputStream(file.toString private var unreadBytes = 0L private var in: DataInputStream = _ @@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new NioBufferedFileInputStream(file)) + in = new DataInputStream(serMgr.wrapForEncryption( +new NioBufferedFileInputStream(file))) } if (unreadBytes > 0) { @@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu private[python] case class HybridRowQueue( memManager: TaskMemoryManager, tempDir: File, -numFields: Int) +numFields: Int, +serMgr: SerializerManager) extends MemoryConsumer(memManager) with RowQueue { // Each buffer should have at least one row @@ -212,7 +219,7 @@ private[python] case class HybridRowQueue( } private def createDiskQueue(): RowQueue = { -DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) +DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr) } private def createNewQueue(required: Long): RowQueue = { @@ -279,3 +286,9 @@ private[python] case class HybridRowQueue( } } } + +private[python] object HybridRowQueue { + def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): HybridRowQueue = { +HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager) + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/8f5a5a9e/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33c..1ec9986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager}
spark git commit: [HOTFIX] fix lint-java
Repository: spark Updated Branches: refs/heads/branch-2.3 575fea120 -> f3bbb7ceb [HOTFIX] fix lint-java Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f3bbb7ce Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f3bbb7ce Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f3bbb7ce Branch: refs/heads/branch-2.3 Commit: f3bbb7ceb9ae8038c3612f1fe5b8b44f0652711a Parents: 575fea1 Author: Imran Rashid Authored: Thu Sep 13 09:47:45 2018 -0500 Committer: Imran Rashid Committed: Thu Sep 13 09:47:45 2018 -0500 -- .../java/org/apache/spark/network/shuffle/DownloadFile.java| 6 +++--- .../spark/network/shuffle/DownloadFileWritableChannel.java | 3 +-- 2 files changed, 4 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f3bbb7ce/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java -- diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java index 76c2d02..633622b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java @@ -32,16 +32,16 @@ public interface DownloadFile { * @return true if and only if the file or directory is * successfully deleted; false otherwise */ - public boolean delete(); + boolean delete(); /** * A channel for writing data to the file. This special channel allows access to the data for * reading, after the channel is closed, via {@link DownloadFileWritableChannel#closeAndRead()}. */ - public DownloadFileWritableChannel openForWriting() throws IOException; + DownloadFileWritableChannel openForWriting() throws IOException; /** * The path of the file, intended only for debug purposes. */ - public String path(); + String path(); } http://git-wip-us.apache.org/repos/asf/spark/blob/f3bbb7ce/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java -- diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java index acf194c..dbbbac4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java @@ -19,7 +19,6 @@ package org.apache.spark.network.shuffle; import org.apache.spark.network.buffer.ManagedBuffer; -import java.io.OutputStream; import java.nio.channels.WritableByteChannel; /** @@ -27,5 +26,5 @@ import java.nio.channels.WritableByteChannel; * after the writer has been closed. Used with DownloadFile and DownloadFileManager. */ public interface DownloadFileWritableChannel extends WritableByteChannel { - public ManagedBuffer closeAndRead(); + ManagedBuffer closeAndRead(); } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[1/4] spark git commit: [CORE] Updates to remote cache reads
Repository: spark Updated Branches: refs/heads/branch-2.3 9ac9f36c4 -> 575fea120 [CORE] Updates to remote cache reads Covered by tests in DistributedSuite Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/575fea12 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/575fea12 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/575fea12 Branch: refs/heads/branch-2.3 Commit: 575fea120e25249716e3f680396580c5f9e26b5b Parents: 6d742d1 Author: Imran Rashid Authored: Wed Aug 22 16:38:28 2018 -0500 Committer: Imran Rashid Committed: Thu Sep 13 09:19:56 2018 -0500 -- .../spark/network/buffer/ManagedBuffer.java | 5 +- .../spark/network/shuffle/DownloadFile.java | 47 ++ .../network/shuffle/DownloadFileManager.java| 36 .../shuffle/DownloadFileWritableChannel.java| 31 +++ .../network/shuffle/ExternalShuffleClient.java | 4 +- .../network/shuffle/OneForOneBlockFetcher.java | 28 +++--- .../spark/network/shuffle/ShuffleClient.java| 4 +- .../network/shuffle/SimpleDownloadFile.java | 91 .../spark/network/shuffle/TempFileManager.java | 36 .../spark/network/BlockTransferService.scala| 6 +- .../netty/NettyBlockTransferService.scala | 4 +- .../org/apache/spark/storage/BlockManager.scala | 78 ++--- .../org/apache/spark/storage/DiskStore.scala| 16 .../storage/ShuffleBlockFetcherIterator.scala | 21 +++-- .../spark/storage/BlockManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 6 +- 16 files changed, 329 insertions(+), 92 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/575fea12/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java -- diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index 1861f8d..2d573f5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -36,7 +36,10 @@ import java.nio.ByteBuffer; */ public abstract class ManagedBuffer { - /** Number of bytes of the data. */ + /** + * Number of bytes of the data. If this buffer will decrypt for all of the views into the data, + * this is the size of the decrypted data. + */ public abstract long size(); /** http://git-wip-us.apache.org/repos/asf/spark/blob/575fea12/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java -- diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java new file mode 100644 index 000..76c2d02 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; + +/** + * A handle on the file used when fetching remote data to disk. Used to ensure the lifecycle of + * writing the data, reading it back, and then cleaning it up is followed. Specific implementations + * may also handle encryption. The data can be read only via DownloadFileWritableChannel, + * which ensures data is not read until after the writer is closed. + */ +public interface DownloadFile { + /** + * Delete the file. + * + * @return true if and only if the file or directory is + * successfully deleted; false otherwise + */ + public boolean delete(); + + /** + * A channel for writing data to the file. This special channel allows access to the data for + * reading, after the channel is
[3/4] spark git commit: [PYSPARK][SQL] Updates to RowQueue
[PYSPARK][SQL] Updates to RowQueue Tested with updates to RowQueueSuite Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6d742d1b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6d742d1b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6d742d1b Branch: refs/heads/branch-2.3 Commit: 6d742d1bd71aa3803dce91a830b37284cb18cf70 Parents: 09dd34c Author: Imran Rashid Authored: Thu Sep 6 12:11:47 2018 -0500 Committer: Imran Rashid Committed: Thu Sep 13 09:19:56 2018 -0500 -- .../spark/sql/execution/python/RowQueue.scala | 27 ++- .../sql/execution/python/RowQueueSuite.scala| 28 +++- 2 files changed, 41 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6d742d1b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index e2fa6e7..d2820ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -21,9 +21,10 @@ import java.io._ import com.google.common.io.Closeables -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.memory.MemoryBlock @@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any * reader has begun reading from the queue. */ -private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { - private var out = new DataOutputStream( -new BufferedOutputStream(new FileOutputStream(file.toString))) +private[python] case class DiskRowQueue( +file: File, +fields: Int, +serMgr: SerializerManager) extends RowQueue { + + private var out = new DataOutputStream(serMgr.wrapForEncryption( +new BufferedOutputStream(new FileOutputStream(file.toString private var unreadBytes = 0L private var in: DataInputStream = _ @@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new NioBufferedFileInputStream(file)) + in = new DataInputStream(serMgr.wrapForEncryption( +new NioBufferedFileInputStream(file))) } if (unreadBytes > 0) { @@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu private[python] case class HybridRowQueue( memManager: TaskMemoryManager, tempDir: File, -numFields: Int) +numFields: Int, +serMgr: SerializerManager) extends MemoryConsumer(memManager) with RowQueue { // Each buffer should have at least one row @@ -212,7 +219,7 @@ private[python] case class HybridRowQueue( } private def createDiskQueue(): RowQueue = { -DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) +DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr) } private def createNewQueue(required: Long): RowQueue = { @@ -279,3 +286,9 @@ private[python] case class HybridRowQueue( } } } + +private[python] object HybridRowQueue { + def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): HybridRowQueue = { +HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager) + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/6d742d1b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33c..1ec9986 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager}
[2/4] spark git commit: [PYSPARK] Updates to pyspark broadcast
[PYSPARK] Updates to pyspark broadcast Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/09dd34cb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/09dd34cb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/09dd34cb Branch: refs/heads/branch-2.3 Commit: 09dd34cb1706f2477a89174d6a1a0f17ed5b0a65 Parents: a2a54a5 Author: Imran Rashid Authored: Mon Aug 13 21:35:34 2018 -0500 Committer: Imran Rashid Committed: Thu Sep 13 09:19:56 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 297 --- .../apache/spark/api/python/PythonRunner.scala | 52 +++- .../spark/api/python/PythonRDDSuite.scala | 23 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/broadcast.py | 58 +++- python/pyspark/context.py | 49 ++- python/pyspark/serializers.py | 58 python/pyspark/test_broadcast.py| 126 python/pyspark/test_serializers.py | 90 ++ python/pyspark/worker.py| 24 +- 10 files changed, 695 insertions(+), 84 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/09dd34cb/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8bc0ff7..5e6bd96 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.Promise +import scala.concurrent.duration.Duration import scala.language.existentials -import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -168,27 +171,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { -val file = new DataInputStream(new FileInputStream(filename)) +readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + } + + def readRDDFromInputStream( + sc: SparkContext, + in: InputStream, + parallelism: Int): JavaRDD[Array[Byte]] = { +val din = new DataInputStream(in) try { val objs = new mutable.ArrayBuffer[Array[Byte]] try { while (true) { - val length = file.readInt() + val length = din.readInt() val obj = new Array[Byte](length) - file.readFully(obj) + din.readFully(obj) objs += obj } } catch { case eof: EOFException => // No-op } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) } finally { - file.close() + din.close() } } - def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { -sc.broadcast(new PythonBroadcast(path)) + def setupBroadcast(path: String): PythonBroadcast = { +new PythonBroadcast(path) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -398,34 +408,15 @@ private[spark] object PythonRDD extends Logging { * data collected from this job, and the secret for authentication. */ def serveIterator(items: Iterator[_], threadName: String): Array[Any] = { -val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) -// Close the socket if no connection in 15 seconds -serverSocket.setSoTimeout(15000) - -new Thread(threadName) { - setDaemon(true) - override def run() { -try { - val sock = serverSocket.accept() - authHelper.authClient(sock) - - val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) - Utils.tryWithSafeFinally { -writeIteratorToStream(items, out) - } { -out.close() -sock.close() - } -} catch { - case NonFatal(e) => -
[4/4] spark git commit: [SPARK-25253][PYSPARK] Refactor local connection & auth code
[SPARK-25253][PYSPARK] Refactor local connection & auth code This eliminates some duplication in the code to connect to a server on localhost to talk directly to the jvm. Also it gives consistent ipv6 and error handling. Two other incidental changes, that shouldn't matter: 1) python barrier tasks perform authentication immediately (rather than waiting for the BARRIER_FUNCTION indicator) 2) for `rdd._load_from_socket`, the timeout is only increased after authentication. Closes #22247 from squito/py_connection_refactor. Authored-by: Imran Rashid Signed-off-by: hyukjinkwon (cherry picked from commit 38391c9aa8a88fcebb337934f30298a32d91596b) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a2a54a5f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a2a54a5f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a2a54a5f Branch: refs/heads/branch-2.3 Commit: a2a54a5f49364a1825932c9f04eb0ff82dd7d465 Parents: 9ac9f36 Author: Imran Rashid Authored: Wed Aug 29 09:47:38 2018 +0800 Committer: Imran Rashid Committed: Thu Sep 13 09:19:56 2018 -0500 -- python/pyspark/java_gateway.py | 32 +++- python/pyspark/rdd.py | 24 ++-- python/pyspark/worker.py | 7 ++- 3 files changed, 35 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a2a54a5f/python/pyspark/java_gateway.py -- diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0afbe9d..eed866d 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -134,7 +134,7 @@ def launch_gateway(conf=None): return gateway -def do_server_auth(conn, auth_secret): +def _do_server_auth(conn, auth_secret): """ Performs the authentication protocol defined by the SocketAuthHelper class on the given file-like object 'conn'. @@ -145,3 +145,33 @@ def do_server_auth(conn, auth_secret): if reply != "ok": conn.close() raise Exception("Unexpected reply from iterator server.") + + +def local_connect_and_auth(port, auth_secret): +""" +Connect to local host, authenticate with it, and return a (sockfile,sock) for that connection. +Handles IPV4 & IPV6, does some error handling. +:param port +:param auth_secret +:return: a tuple with (sockfile, sock) +""" +sock = None +errors = [] +# Support for both IPv4 and IPv6. +# On most of IPv6-ready systems, IPv6 will take precedence. +for res in socket.getaddrinfo("127.0.0.1", port, socket.AF_UNSPEC, socket.SOCK_STREAM): +af, socktype, proto, _, sa = res +try: +sock = socket.socket(af, socktype, proto) +sock.settimeout(15) +sock.connect(sa) +sockfile = sock.makefile("rwb", 65536) +_do_server_auth(sockfile, auth_secret) +return (sockfile, sock) +except socket.error as e: +emsg = _exception_message(e) +errors.append("tried to connect to %s, but an error occured: %s" % (sa, emsg)) +sock.close() +sock = None +else: +raise Exception("could not open socket: %s" % errors) http://git-wip-us.apache.org/repos/asf/spark/blob/a2a54a5f/python/pyspark/rdd.py -- diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index eadb5ab..3f6a8e6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -39,7 +39,7 @@ if sys.version > '3': else: from itertools import imap as map, ifilter as filter -from pyspark.java_gateway import do_server_auth +from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \ @@ -139,30 +139,10 @@ def _parse_memory(s): def _load_from_socket(sock_info, serializer): -port, auth_secret = sock_info -sock = None -# Support for both IPv4 and IPv6. -# On most of IPv6-ready systems, IPv6 will take precedence. -for res in socket.getaddrinfo("localhost", port, socket.AF_UNSPEC, socket.SOCK_STREAM): -af, socktype, proto, canonname, sa = res -sock = socket.socket(af, socktype, proto) -try: -sock.settimeout(15) -sock.connect(sa) -except socket.error: -sock.close() -sock = None -continue -break -if not sock: -raise Exception("could not open socket") +(sockfile, sock) = local_connect_and_auth(*sock_info) # The RDD
spark git commit: [PYSPARK] Updates to Accumulators
Repository: spark Updated Branches: refs/heads/branch-2.1 a3eb07db3 -> b2e0f68f6 [PYSPARK] Updates to Accumulators (cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b2e0f68f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b2e0f68f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b2e0f68f Branch: refs/heads/branch-2.1 Commit: b2e0f68f615cbe2cf74f9813ece76c311fe8e911 Parents: a3eb07d Author: LucaCanali Authored: Wed Jul 18 23:19:02 2018 +0200 Committer: Imran Rashid Committed: Fri Aug 3 16:30:40 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 12 +++-- python/pyspark/accumulators.py | 53 +++- python/pyspark/context.py | 5 +- 3 files changed, 53 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b2e0f68f/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b1190b9..de548e8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -886,8 +886,9 @@ class BytesToString extends org.apache.spark.api.java.function.Function[Array[By */ private[spark] class PythonAccumulatorV2( @transient private val serverHost: String, -private val serverPort: Int) - extends CollectionAccumulator[Array[Byte]] { +private val serverPort: Int, +private val secretToken: String) + extends CollectionAccumulator[Array[Byte]] with Logging{ Utils.checkHost(serverHost, "Expected hostname") @@ -902,12 +903,17 @@ private[spark] class PythonAccumulatorV2( private def openSocket(): Socket = synchronized { if (socket == null || socket.isClosed) { socket = new Socket(serverHost, serverPort) + logInfo(s"Connected to AccumulatorServer at host: $serverHost port: $serverPort") + // send the secret just for the initial authentication when opening a new connection + socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8)) } socket } // Need to override so the types match with PythonFunction - override def copyAndReset(): PythonAccumulatorV2 = new PythonAccumulatorV2(serverHost, serverPort) + override def copyAndReset(): PythonAccumulatorV2 = { +new PythonAccumulatorV2(serverHost, serverPort, secretToken) + } override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): Unit = synchronized { val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2] http://git-wip-us.apache.org/repos/asf/spark/blob/b2e0f68f/python/pyspark/accumulators.py -- diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 6ef8cf5..bc0be07 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -228,20 +228,49 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler): def handle(self): from pyspark.accumulators import _accumulatorRegistry -while not self.server.server_shutdown: -# Poll every 1 second for new data -- don't block in case of shutdown. -r, _, _ = select.select([self.rfile], [], [], 1) -if self.rfile in r: -num_updates = read_int(self.rfile) -for _ in range(num_updates): -(aid, update) = pickleSer._read_with_length(self.rfile) -_accumulatorRegistry[aid] += update -# Write a byte in acknowledgement -self.wfile.write(struct.pack("!b", 1)) +auth_token = self.server.auth_token + +def poll(func): +while not self.server.server_shutdown: +# Poll every 1 second for new data -- don't block in case of shutdown. +r, _, _ = select.select([self.rfile], [], [], 1) +if self.rfile in r: +if func(): +break + +def accum_updates(): +num_updates = read_int(self.rfile) +for _ in range(num_updates): +(aid, update) = pickleSer._read_with_length(self.rfile) +_accumulatorRegistry[aid] += update +# Write a byte in acknowledgement +self.wfile.write(struct.pack("!b", 1)) +return False + +def authenticate_and_accum_updates(): +received_token = self.rfile.read(len(auth_token)) +if
spark git commit: [PYSPARK] Updates to Accumulators
Repository: spark Updated Branches: refs/heads/branch-2.2 22ce8051f -> a5624c7ae [PYSPARK] Updates to Accumulators (cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a5624c7a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a5624c7a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a5624c7a Branch: refs/heads/branch-2.2 Commit: a5624c7ae29d6d49117dd78642879bf978212d30 Parents: 22ce805 Author: LucaCanali Authored: Wed Jul 18 23:19:02 2018 +0200 Committer: Imran Rashid Committed: Fri Aug 3 14:49:38 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 12 +++-- python/pyspark/accumulators.py | 53 +++- python/pyspark/context.py | 5 +- 3 files changed, 53 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a5624c7a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 0662792..7b5a179 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -900,8 +900,9 @@ class BytesToString extends org.apache.spark.api.java.function.Function[Array[By */ private[spark] class PythonAccumulatorV2( @transient private val serverHost: String, -private val serverPort: Int) - extends CollectionAccumulator[Array[Byte]] { +private val serverPort: Int, +private val secretToken: String) + extends CollectionAccumulator[Array[Byte]] with Logging{ Utils.checkHost(serverHost, "Expected hostname") @@ -916,12 +917,17 @@ private[spark] class PythonAccumulatorV2( private def openSocket(): Socket = synchronized { if (socket == null || socket.isClosed) { socket = new Socket(serverHost, serverPort) + logInfo(s"Connected to AccumulatorServer at host: $serverHost port: $serverPort") + // send the secret just for the initial authentication when opening a new connection + socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8)) } socket } // Need to override so the types match with PythonFunction - override def copyAndReset(): PythonAccumulatorV2 = new PythonAccumulatorV2(serverHost, serverPort) + override def copyAndReset(): PythonAccumulatorV2 = { +new PythonAccumulatorV2(serverHost, serverPort, secretToken) + } override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): Unit = synchronized { val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2] http://git-wip-us.apache.org/repos/asf/spark/blob/a5624c7a/python/pyspark/accumulators.py -- diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 6ef8cf5..bc0be07 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -228,20 +228,49 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler): def handle(self): from pyspark.accumulators import _accumulatorRegistry -while not self.server.server_shutdown: -# Poll every 1 second for new data -- don't block in case of shutdown. -r, _, _ = select.select([self.rfile], [], [], 1) -if self.rfile in r: -num_updates = read_int(self.rfile) -for _ in range(num_updates): -(aid, update) = pickleSer._read_with_length(self.rfile) -_accumulatorRegistry[aid] += update -# Write a byte in acknowledgement -self.wfile.write(struct.pack("!b", 1)) +auth_token = self.server.auth_token + +def poll(func): +while not self.server.server_shutdown: +# Poll every 1 second for new data -- don't block in case of shutdown. +r, _, _ = select.select([self.rfile], [], [], 1) +if self.rfile in r: +if func(): +break + +def accum_updates(): +num_updates = read_int(self.rfile) +for _ in range(num_updates): +(aid, update) = pickleSer._read_with_length(self.rfile) +_accumulatorRegistry[aid] += update +# Write a byte in acknowledgement +self.wfile.write(struct.pack("!b", 1)) +return False + +def authenticate_and_accum_updates(): +received_token = self.rfile.read(len(auth_token)) +if
spark git commit: [PYSPARK] Updates to Accumulators
Repository: spark Updated Branches: refs/heads/branch-2.3 5b187a85a -> 8080c937d [PYSPARK] Updates to Accumulators (cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8080c937 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8080c937 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8080c937 Branch: refs/heads/branch-2.3 Commit: 8080c937d3752aee2fd36f0045a057f7130f6fe4 Parents: 5b187a8 Author: LucaCanali Authored: Wed Jul 18 23:19:02 2018 +0200 Committer: Imran Rashid Committed: Thu Aug 2 21:05:03 2018 -0500 -- .../org/apache/spark/api/python/PythonRDD.scala | 12 +++-- python/pyspark/accumulators.py | 53 +++- python/pyspark/context.py | 5 +- 3 files changed, 53 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8080c937/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index a1ee2f7..8bc0ff7 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -586,8 +586,9 @@ class BytesToString extends org.apache.spark.api.java.function.Function[Array[By */ private[spark] class PythonAccumulatorV2( @transient private val serverHost: String, -private val serverPort: Int) - extends CollectionAccumulator[Array[Byte]] { +private val serverPort: Int, +private val secretToken: String) + extends CollectionAccumulator[Array[Byte]] with Logging{ Utils.checkHost(serverHost) @@ -602,12 +603,17 @@ private[spark] class PythonAccumulatorV2( private def openSocket(): Socket = synchronized { if (socket == null || socket.isClosed) { socket = new Socket(serverHost, serverPort) + logInfo(s"Connected to AccumulatorServer at host: $serverHost port: $serverPort") + // send the secret just for the initial authentication when opening a new connection + socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8)) } socket } // Need to override so the types match with PythonFunction - override def copyAndReset(): PythonAccumulatorV2 = new PythonAccumulatorV2(serverHost, serverPort) + override def copyAndReset(): PythonAccumulatorV2 = { +new PythonAccumulatorV2(serverHost, serverPort, secretToken) + } override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): Unit = synchronized { val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2] http://git-wip-us.apache.org/repos/asf/spark/blob/8080c937/python/pyspark/accumulators.py -- diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 6ef8cf5..bc0be07 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -228,20 +228,49 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler): def handle(self): from pyspark.accumulators import _accumulatorRegistry -while not self.server.server_shutdown: -# Poll every 1 second for new data -- don't block in case of shutdown. -r, _, _ = select.select([self.rfile], [], [], 1) -if self.rfile in r: -num_updates = read_int(self.rfile) -for _ in range(num_updates): -(aid, update) = pickleSer._read_with_length(self.rfile) -_accumulatorRegistry[aid] += update -# Write a byte in acknowledgement -self.wfile.write(struct.pack("!b", 1)) +auth_token = self.server.auth_token + +def poll(func): +while not self.server.server_shutdown: +# Poll every 1 second for new data -- don't block in case of shutdown. +r, _, _ = select.select([self.rfile], [], [], 1) +if self.rfile in r: +if func(): +break + +def accum_updates(): +num_updates = read_int(self.rfile) +for _ in range(num_updates): +(aid, update) = pickleSer._read_with_length(self.rfile) +_accumulatorRegistry[aid] += update +# Write a byte in acknowledgement +self.wfile.write(struct.pack("!b", 1)) +return False + +def authenticate_and_accum_updates(): +received_token = self.rfile.read(len(auth_token)) +if isinstance(received_token, bytes): +
spark git commit: [SPARK-24801][CORE] Avoid memory waste by empty byte[] arrays in SaslEncryption$EncryptedMessage
Repository: spark Updated Branches: refs/heads/master fa09d9192 -> 094aa5971 [SPARK-24801][CORE] Avoid memory waste by empty byte[] arrays in SaslEncryption$EncryptedMessage ## What changes were proposed in this pull request? Initialize SaslEncryption$EncryptedMessage.byteChannel lazily, so that empty, not yet used instances of ByteArrayWritableChannel referenced by this field don't use up memory. I analyzed a heap dump from Yarn Node Manager where this code is used, and found that there are over 40,000 of the above objects in memory, each with a big empty byte[] array. The reason they are all there is because of Netty queued up a large number of messages in memory before transferTo() is called. There is a small number of netty ChannelOutboundBuffer objects, and then collectively , via linked lists starting from their flushedEntry data fields, they end up referencing over 40K ChannelOutboundBuffer$Entry objects, which ultimately reference EncryptedMessage objects. ## How was this patch tested? Ran all the tests locally. Author: Misha Dmitriev Closes #21811 from countmdm/misha/spark-24801. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/094aa597 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/094aa597 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/094aa597 Branch: refs/heads/master Commit: 094aa597155dfcbf41a2490c9e462415e3824901 Parents: fa09d91 Author: Misha Dmitriev Authored: Thu Jul 26 22:15:12 2018 -0500 Committer: Imran Rashid Committed: Thu Jul 26 22:15:12 2018 -0500 -- .../org/apache/spark/network/sasl/SaslEncryption.java | 10 +++--- 1 file changed, 7 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/094aa597/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java -- diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java index 3ac9081..d3b2a33 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java @@ -135,13 +135,14 @@ class SaslEncryption { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; +private final int maxOutboundBlockSize; /** * A channel used to buffer input data for encryption. The channel has an upper size bound * so that if the input is larger than the allowed buffer, it will be broken into multiple - * chunks. + * chunks. Made non-final to enable lazy initialization, which saves memory. */ -private final ByteArrayWritableChannel byteChannel; +private ByteArrayWritableChannel byteChannel; private ByteBuf currentHeader; private ByteBuffer currentChunk; @@ -157,7 +158,7 @@ class SaslEncryption { this.isByteBuf = msg instanceof ByteBuf; this.buf = isByteBuf ? (ByteBuf) msg : null; this.region = isByteBuf ? null : (FileRegion) msg; - this.byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize); + this.maxOutboundBlockSize = maxOutboundBlockSize; } /** @@ -292,6 +293,9 @@ class SaslEncryption { } private void nextChunk() throws IOException { + if (byteChannel == null) { +byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize); + } byteChannel.reset(); if (isByteBuf) { int copied = byteChannel.write(buf.nioBuffer()); - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-16630][YARN] Blacklist a node if executors won't launch on it
Repository: spark Updated Branches: refs/heads/master c0cad596b -> b56e9c613 [SPARK-16630][YARN] Blacklist a node if executors won't launch on it ## What changes were proposed in this pull request? This change extends YARN resource allocation handling with blacklisting functionality. This handles cases when node is messed up or misconfigured such that a container won't launch on it. Before this change backlisting only focused on task execution but this change introduces YarnAllocatorBlacklistTracker which tracks allocation failures per host (when enabled via "spark.yarn.blacklist.executor.launch.blacklisting.enabled"). ## How was this patch tested? ### With unit tests Including a new suite: YarnAllocatorBlacklistTrackerSuite. Manually It was tested on a cluster by deleting the Spark jars on one of the node. Behaviour before these changes Starting Spark as: ``` spark2-shell --master yarn --deploy-mode client --num-executors 4 --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" ``` Log is: ``` 18/04/12 06:49:36 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Max number of executor failures (6) reached) 18/04/12 06:49:39 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster with FAILED (diag message: Max number of executor failures (6) reached) 18/04/12 06:49:39 INFO impl.AMRMClientImpl: Waiting for application to be successfully unregistered. 18/04/12 06:49:39 INFO yarn.ApplicationMaster: Deleting staging directory hdfs://apiros-1.gce.test.com:8020/user/systest/.sparkStaging/application_1523459048274_0016 18/04/12 06:49:39 INFO util.ShutdownHookManager: Shutdown hook called ``` Behaviour after these changes Starting Spark as: ``` spark2-shell --master yarn --deploy-mode client --num-executors 4 --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" --conf "spark.yarn.blacklist.executor.launch.blacklisting.enabled=true" ``` And the log is: ``` 18/04/13 05:37:43 INFO yarn.YarnAllocator: Will request 1 executor container(s), each with 1 core(s) and 4505 MB memory (including 409 MB of overhead) 18/04/13 05:37:43 INFO yarn.YarnAllocator: Submitted 1 unlocalized container requests. 18/04/13 05:37:43 INFO yarn.YarnAllocator: Launching container container_1523459048274_0025_01_08 on host apiros-4.gce.test.com for executor with ID 6 18/04/13 05:37:43 INFO yarn.YarnAllocator: Received 1 containers from YARN, launching executors on 1 of them. 18/04/13 05:37:43 INFO yarn.YarnAllocator: Completed container container_1523459048274_0025_01_07 on host: apiros-4.gce.test.com (state: COMPLETE, exit status: 1) 18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com 18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com) 18/04/13 05:37:43 WARN yarn.YarnAllocator: Container marked as failed: container_1523459048274_0025_01_07 on host: apiros-4.gce.test.com. Exit status: 1. Diagnostics: Exception from container-launch. Container id: container_1523459048274_0025_01_07 Exit code: 1 Stack trace: ExitCodeException exitCode=1: at org.apache.hadoop.util.Shell.runCommand(Shell.java:604) at org.apache.hadoop.util.Shell.run(Shell.java:507) at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:789) at org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` Where the most important part is: ``` 18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com 18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com) ``` And execution was continued (no shutdown called). ### Testing the backlisting of the whole cluster Starting Spark with YARN blacklisting enabled then removing a the Spark core jar one by one from all the cluster nodes. Then executing a simple spark job which fails checking the yarn log the expected exit status is contained: ``` 18/06/15 01:07:10 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Due to executor failures all available nodes are blacklisted) 18/06/15 01:07:13 INFO
spark git commit: [SPARK-24416] Fix configuration specification for killBlacklisted executors
Repository: spark Updated Branches: refs/heads/master 2824f1436 -> 3af1d3e6d [SPARK-24416] Fix configuration specification for killBlacklisted executors ## What changes were proposed in this pull request? spark.blacklist.killBlacklistedExecutors is defined as (Experimental) If set to "true", allow Spark to automatically kill, and attempt to re-create, executors when they are blacklisted. Note that, when an entire node is added to the blacklist, all of the executors on that node will be killed. I presume the killing of blacklisted executors only happens after the stage completes successfully and all tasks have completed or on fetch failures (updateBlacklistForFetchFailure/updateBlacklistForSuccessfulTaskSet). It is confusing because the definition states that the executor will be attempted to be recreated as soon as it is blacklisted. This is not true while the stage is in progress and an executor is blacklisted, it will not attempt to cleanup until the stage finishes. Author: Sanket Chintapalli Closes #21475 from redsanket/SPARK-24416. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3af1d3e6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3af1d3e6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3af1d3e6 Branch: refs/heads/master Commit: 3af1d3e6d95719e15a997877d5ecd3bb40c08b9c Parents: 2824f14 Author: Sanket Chintapalli Authored: Tue Jun 12 13:55:08 2018 -0500 Committer: Imran Rashid Committed: Tue Jun 12 13:55:08 2018 -0500 -- docs/configuration.md | 7 --- 1 file changed, 4 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3af1d3e6/docs/configuration.md -- diff --git a/docs/configuration.md b/docs/configuration.md index 5588c37..6aa7878 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1656,9 +1656,10 @@ Apart from these, the following properties are also available, and may be useful spark.blacklist.killBlacklistedExecutors false -(Experimental) If set to "true", allow Spark to automatically kill, and attempt to re-create, -executors when they are blacklisted. Note that, when an entire node is added to the blacklist, -all of the executors on that node will be killed. +(Experimental) If set to "true", allow Spark to automatically kill the executors +when they are blacklisted on fetch failure or blacklisted for the entire application, +as controlled by spark.blacklist.application.*. Note that, when an entire node is added +to the blacklist, all of the executors on that node will be killed. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets
Repository: spark Updated Branches: refs/heads/branch-2.2 4f1ae3af9 -> 154bbc959 [SPARK-23433][CORE] Late zombie task completions update all tasksets Fetch failure lead to multiple tasksets which are active for a given stage. While there is only one "active" version of the taskset, the earlier attempts can still have running tasks, which can complete successfully. So a task completion needs to update every taskset so that it knows the partition is completed. That way the final active taskset does not try to submit another task for the same partition, and so that it knows when it is completed and when it should be marked as a "zombie". Added a regression test. Author: Imran RashidCloses #21131 from squito/SPARK-23433. (cherry picked from commit 94641fe6cc68e5977dd8663b8f232a287a783acb) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/154bbc95 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/154bbc95 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/154bbc95 Branch: refs/heads/branch-2.2 Commit: 154bbc95989b34b2007022b6082ed17864bbaa32 Parents: 4f1ae3a Author: Imran Rashid Authored: Thu May 3 10:59:18 2018 -0500 Committer: Imran Rashid Committed: Thu May 3 10:59:45 2018 -0500 -- .../spark/scheduler/TaskSchedulerImpl.scala | 14 +++ .../apache/spark/scheduler/TaskSetManager.scala | 20 +++- .../scheduler/TaskSchedulerImplSuite.scala | 104 +++ 3 files changed, 137 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/154bbc95/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 1b6bc91..df6407b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -693,6 +693,20 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } + /** + * Marks the task has completed in all TaskSetManagers for the given stage. + * + * After stage failure and retry, there may be multiple TaskSetManagers for the stage. + * If an earlier attempt of a stage completes a task, we should ensure that the later attempts + * do not also submit those same tasks. That also means that a task completion from an earlier + * attempt can lead to the entire stage getting marked as successful. + */ + private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, partitionId: Int) = { +taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm => + tsm.markPartitionCompleted(partitionId) +} + } + } http://git-wip-us.apache.org/repos/asf/spark/blob/154bbc95/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 2f4e46c..d9515fb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -74,6 +74,8 @@ private[spark] class TaskSetManager( val ser = env.closureSerializer.newInstance() val tasks = taskSet.tasks + private[scheduler] val partitionToIndex = tasks.zipWithIndex +.map { case (t, idx) => t.partitionId -> idx }.toMap val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -149,7 +151,7 @@ private[spark] class TaskSetManager( private[scheduler] val speculatableTasks = new HashSet[Int] // Task index, start and finish time for each task attempt (indexed by task ID) - private val taskInfos = new HashMap[Long, TaskInfo] + private[scheduler] val taskInfos = new HashMap[Long, TaskInfo] // Use a MedianHeap to record durations of successful tasks so we know when to launch // speculative tasks. This is only used when speculation is enabled, to avoid the overhead @@ -744,6 +746,9 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } +// There may be multiple tasksets for this stage -- we let all of them know that the partition +// was completed. This may result in some of the tasksets getting completed. +
spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets
Repository: spark Updated Branches: refs/heads/branch-2.3 61e7bc0c1 -> 8509284e1 [SPARK-23433][CORE] Late zombie task completions update all tasksets Fetch failure lead to multiple tasksets which are active for a given stage. While there is only one "active" version of the taskset, the earlier attempts can still have running tasks, which can complete successfully. So a task completion needs to update every taskset so that it knows the partition is completed. That way the final active taskset does not try to submit another task for the same partition, and so that it knows when it is completed and when it should be marked as a "zombie". Added a regression test. Author: Imran RashidCloses #21131 from squito/SPARK-23433. (cherry picked from commit 94641fe6cc68e5977dd8663b8f232a287a783acb) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8509284e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8509284e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8509284e Branch: refs/heads/branch-2.3 Commit: 8509284e1ec048d5afa87d41071c0429924e45c9 Parents: 61e7bc0 Author: Imran Rashid Authored: Thu May 3 10:59:18 2018 -0500 Committer: Imran Rashid Committed: Thu May 3 10:59:30 2018 -0500 -- .../spark/scheduler/TaskSchedulerImpl.scala | 14 +++ .../apache/spark/scheduler/TaskSetManager.scala | 20 +++- .../scheduler/TaskSchedulerImplSuite.scala | 104 +++ 3 files changed, 137 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8509284e/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0c11806..8e97b3d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -689,6 +689,20 @@ private[spark] class TaskSchedulerImpl( } } + /** + * Marks the task has completed in all TaskSetManagers for the given stage. + * + * After stage failure and retry, there may be multiple TaskSetManagers for the stage. + * If an earlier attempt of a stage completes a task, we should ensure that the later attempts + * do not also submit those same tasks. That also means that a task completion from an earlier + * attempt can lead to the entire stage getting marked as successful. + */ + private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, partitionId: Int) = { +taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm => + tsm.markPartitionCompleted(partitionId) +} + } + } http://git-wip-us.apache.org/repos/asf/spark/blob/8509284e/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c3ed11b..b52e376 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -74,6 +74,8 @@ private[spark] class TaskSetManager( val ser = env.closureSerializer.newInstance() val tasks = taskSet.tasks + private[scheduler] val partitionToIndex = tasks.zipWithIndex +.map { case (t, idx) => t.partitionId -> idx }.toMap val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -154,7 +156,7 @@ private[spark] class TaskSetManager( private[scheduler] val speculatableTasks = new HashSet[Int] // Task index, start and finish time for each task attempt (indexed by task ID) - private val taskInfos = new HashMap[Long, TaskInfo] + private[scheduler] val taskInfos = new HashMap[Long, TaskInfo] // Use a MedianHeap to record durations of successful tasks so we know when to launch // speculative tasks. This is only used when speculation is enabled, to avoid the overhead @@ -755,6 +757,9 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } +// There may be multiple tasksets for this stage -- we let all of them know that the partition +// was completed. This may result in some of the tasksets getting completed. +sched.markPartitionCompletedInAllTaskSets(stageId,
spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets
Repository: spark Updated Branches: refs/heads/master 96a50016b -> 94641fe6c [SPARK-23433][CORE] Late zombie task completions update all tasksets Fetch failure lead to multiple tasksets which are active for a given stage. While there is only one "active" version of the taskset, the earlier attempts can still have running tasks, which can complete successfully. So a task completion needs to update every taskset so that it knows the partition is completed. That way the final active taskset does not try to submit another task for the same partition, and so that it knows when it is completed and when it should be marked as a "zombie". Added a regression test. Author: Imran RashidCloses #21131 from squito/SPARK-23433. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/94641fe6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/94641fe6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/94641fe6 Branch: refs/heads/master Commit: 94641fe6cc68e5977dd8663b8f232a287a783acb Parents: 96a5001 Author: Imran Rashid Authored: Thu May 3 10:59:18 2018 -0500 Committer: Imran Rashid Committed: Thu May 3 10:59:18 2018 -0500 -- .../spark/scheduler/TaskSchedulerImpl.scala | 14 +++ .../apache/spark/scheduler/TaskSetManager.scala | 20 +++- .../scheduler/TaskSchedulerImplSuite.scala | 104 +++ 3 files changed, 137 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/94641fe6/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0c11806..8e97b3d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -689,6 +689,20 @@ private[spark] class TaskSchedulerImpl( } } + /** + * Marks the task has completed in all TaskSetManagers for the given stage. + * + * After stage failure and retry, there may be multiple TaskSetManagers for the stage. + * If an earlier attempt of a stage completes a task, we should ensure that the later attempts + * do not also submit those same tasks. That also means that a task completion from an earlier + * attempt can lead to the entire stage getting marked as successful. + */ + private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, partitionId: Int) = { +taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm => + tsm.markPartitionCompleted(partitionId) +} + } + } http://git-wip-us.apache.org/repos/asf/spark/blob/94641fe6/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 8a96a76..195fc80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -73,6 +73,8 @@ private[spark] class TaskSetManager( val ser = env.closureSerializer.newInstance() val tasks = taskSet.tasks + private[scheduler] val partitionToIndex = tasks.zipWithIndex +.map { case (t, idx) => t.partitionId -> idx }.toMap val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -153,7 +155,7 @@ private[spark] class TaskSetManager( private[scheduler] val speculatableTasks = new HashSet[Int] // Task index, start and finish time for each task attempt (indexed by task ID) - private val taskInfos = new HashMap[Long, TaskInfo] + private[scheduler] val taskInfos = new HashMap[Long, TaskInfo] // Use a MedianHeap to record durations of successful tasks so we know when to launch // speculative tasks. This is only used when speculation is enabled, to avoid the overhead @@ -754,6 +756,9 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } +// There may be multiple tasksets for this stage -- we let all of them know that the partition +// was completed. This may result in some of the tasksets getting completed. +sched.markPartitionCompletedInAllTaskSets(stageId, tasks(index).partitionId) // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the //
spark git commit: [SPARK-23888][CORE] correct the comment of hasAttemptOnHost()
Repository: spark Updated Branches: refs/heads/master e82cb6834 -> c8f3ac69d [SPARK-23888][CORE] correct the comment of hasAttemptOnHost() TaskSetManager.hasAttemptOnHost had a misleading comment. The comment said that it only checked for running tasks, but really it checked for any tasks that might have run in the past as well. This updates to line up with the implementation. Author: wuyiCloses #20998 from Ngone51/SPARK-23888. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c8f3ac69 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c8f3ac69 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c8f3ac69 Branch: refs/heads/master Commit: c8f3ac69d176bd10b8de1c147b6903a247943d51 Parents: e82cb68 Author: wuyi Authored: Mon Apr 23 15:35:45 2018 -0500 Committer: Imran Rashid Committed: Mon Apr 23 15:35:50 2018 -0500 -- .../src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c8f3ac69/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d958658..8a96a76 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -287,7 +287,7 @@ private[spark] class TaskSetManager( None } - /** Check whether a task is currently running an attempt on a given host */ + /** Check whether a task once ran an attempt on a given host */ private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = { taskAttempts(taskIndex).exists(_.host == host) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure
Repository: spark Updated Branches: refs/heads/branch-2.3 32bec6ca3 -> 7fb11176f [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure ## What changes were proposed in this pull request? Thereâs a miswrite in BlacklistTracker's updateBlacklistForFetchFailure: ``` val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) blacklistedExecsOnNode += exec ``` where first **exec** should be **host**. ## How was this patch tested? adjust existed test. Author: wuyiCloses #21104 from Ngone51/SPARK-24021. (cherry picked from commit 0deaa5251326a32a3d2d2b8851193ca926303972) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7fb11176 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7fb11176 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7fb11176 Branch: refs/heads/branch-2.3 Commit: 7fb11176f285b4de47e61511c09acbbb79e5c44c Parents: 32bec6c Author: wuyi Authored: Thu Apr 19 09:00:33 2018 -0500 Committer: Imran Rashid Committed: Thu Apr 19 09:00:46 2018 -0500 -- .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 2 +- .../org/apache/spark/scheduler/BlacklistTrackerSuite.scala | 5 + 2 files changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7fb11176/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 952598f..30cf75d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -210,7 +210,7 @@ private[scheduler] class BlacklistTracker ( updateNextExpiryTime() killBlacklistedExecutor(exec) -val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) +val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(host, HashSet[String]()) blacklistedExecsOnNode += exec } } http://git-wip-us.apache.org/repos/asf/spark/blob/7fb11176/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 88a57c5..00867ef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -574,6 +574,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M verify(allocationClientMock, never).killExecutors(any(), any(), any(), any()) verify(allocationClientMock, never).killExecutorsOnHost(any()) +assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) +assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) + // Enable auto-kill. Blacklist an executor and make sure killExecutors is called. conf.set(config.BLACKLIST_KILL_ENABLED, true) blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) @@ -589,6 +592,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty) +assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) +assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) // Enable external shuffle service to see if all the executors on this node will be killed. conf.set(config.SHUFFLE_SERVICE_ENABLED, true) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure
Repository: spark Updated Branches: refs/heads/master d96c3e33c -> 0deaa5251 [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure ## What changes were proposed in this pull request? Thereâs a miswrite in BlacklistTracker's updateBlacklistForFetchFailure: ``` val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) blacklistedExecsOnNode += exec ``` where first **exec** should be **host**. ## How was this patch tested? adjust existed test. Author: wuyiCloses #21104 from Ngone51/SPARK-24021. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0deaa525 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0deaa525 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0deaa525 Branch: refs/heads/master Commit: 0deaa5251326a32a3d2d2b8851193ca926303972 Parents: d96c3e3 Author: wuyi Authored: Thu Apr 19 09:00:33 2018 -0500 Committer: Imran Rashid Committed: Thu Apr 19 09:00:33 2018 -0500 -- .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 2 +- .../org/apache/spark/scheduler/BlacklistTrackerSuite.scala | 5 + 2 files changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0deaa525/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 952598f..30cf75d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -210,7 +210,7 @@ private[scheduler] class BlacklistTracker ( updateNextExpiryTime() killBlacklistedExecutor(exec) -val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) +val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(host, HashSet[String]()) blacklistedExecsOnNode += exec } } http://git-wip-us.apache.org/repos/asf/spark/blob/0deaa525/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 06d7afa..96c8404 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -574,6 +574,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M verify(allocationClientMock, never).killExecutors(any(), any(), any(), any()) verify(allocationClientMock, never).killExecutorsOnHost(any()) +assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) +assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) + // Enable auto-kill. Blacklist an executor and make sure killExecutors is called. conf.set(config.BLACKLIST_KILL_ENABLED, true) blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) @@ -589,6 +592,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty) +assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) +assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) // Enable external shuffle service to see if all the executors on this node will be killed. conf.set(config.SHUFFLE_SERVICE_ENABLED, true) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks
Repository: spark Updated Branches: refs/heads/branch-2.3 564019b92 -> 6b99d5bc3 [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks ## What changes were proposed in this pull request? SparkContext submitted a map stage from `submitMapStage` to `DAGScheduler`, `markMapStageJobAsFinished` is called only in (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L933 and https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1314); But think about below scenario: 1. stage0 and stage1 are all `ShuffleMapStage` and stage1 depends on stage0; 2. We submit stage1 by `submitMapStage`; 3. When stage 1 running, `FetchFailed` happened, stage0 and stage1 got resubmitted as stage0_1 and stage1_1; 4. When stage0_1 running, speculated tasks in old stage1 come as succeeded, but stage1 is not inside `runningStages`. So even though all splits(including the speculated tasks) in stage1 succeeded, job listener in stage1 will not be called; 5. stage0_1 finished, stage1_1 starts running. When `submitMissingTasks`, there is no missing tasks. But in current code, job listener is not triggered. We should call the job listener for map stage in `5`. ## How was this patch tested? Not added yet. Author: jinxing (cherry picked from commit 3990daaf3b6ca2c5a9f7790030096262efb12cb2) Author: jinxingCloses #21085 from squito/cp. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6b99d5bc Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6b99d5bc Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6b99d5bc Branch: refs/heads/branch-2.3 Commit: 6b99d5bc3f3898a0aff30468a623a3f64bb20b62 Parents: 564019b Author: jinxing Authored: Tue Apr 17 15:53:29 2018 -0500 Committer: Imran Rashid Committed: Tue Apr 17 15:53:29 2018 -0500 -- .../apache/spark/scheduler/DAGScheduler.scala | 33 +++-- .../spark/scheduler/DAGSchedulerSuite.scala | 52 2 files changed, 70 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6b99d5bc/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8c46a84..78b6b34 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1092,17 +1092,16 @@ class DAGScheduler( // the stage as completed here in case there are no tasks to run markStageAsFinished(stage, None) - val debugString = stage match { + stage match { case stage: ShuffleMapStage => - s"Stage ${stage} is actually done; " + -s"(available: ${stage.isAvailable}," + -s"available outputs: ${stage.numAvailableOutputs}," + -s"partitions: ${stage.numPartitions})" + logDebug(s"Stage ${stage} is actually done; " + + s"(available: ${stage.isAvailable}," + + s"available outputs: ${stage.numAvailableOutputs}," + + s"partitions: ${stage.numPartitions})") + markMapStageJobsAsFinished(stage) case stage : ResultStage => - s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" + logDebug(s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})") } - logDebug(debugString) - submitWaitingChildStages(stage) } } @@ -1307,13 +1306,7 @@ class DAGScheduler( shuffleStage.findMissingPartitions().mkString(", ")) submitStage(shuffleStage) } else { -// Mark any map-stage jobs waiting on this stage as finished -if (shuffleStage.mapStageJobs.nonEmpty) { - val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep) - for (job <- shuffleStage.mapStageJobs) { -markMapStageJobAsFinished(job, stats) - } -} +markMapStageJobsAsFinished(shuffleStage) submitWaitingChildStages(shuffleStage) } } @@ -1433,6 +1426,16 @@ class DAGScheduler( } } + private[scheduler] def markMapStageJobsAsFinished(shuffleStage: ShuffleMapStage): Unit = { +// Mark any map-stage jobs waiting on this stage as finished +if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) { + val stats =
spark git commit: [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks
Repository: spark Updated Branches: refs/heads/master ed4101d29 -> 3990daaf3 [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks ## What changes were proposed in this pull request? SparkContext submitted a map stage from `submitMapStage` to `DAGScheduler`, `markMapStageJobAsFinished` is called only in (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L933 and https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1314); But think about below scenario: 1. stage0 and stage1 are all `ShuffleMapStage` and stage1 depends on stage0; 2. We submit stage1 by `submitMapStage`; 3. When stage 1 running, `FetchFailed` happened, stage0 and stage1 got resubmitted as stage0_1 and stage1_1; 4. When stage0_1 running, speculated tasks in old stage1 come as succeeded, but stage1 is not inside `runningStages`. So even though all splits(including the speculated tasks) in stage1 succeeded, job listener in stage1 will not be called; 5. stage0_1 finished, stage1_1 starts running. When `submitMissingTasks`, there is no missing tasks. But in current code, job listener is not triggered. We should call the job listener for map stage in `5`. ## How was this patch tested? Not added yet. Author: jinxingCloses #21019 from jinxing64/SPARK-23948. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3990daaf Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3990daaf Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3990daaf Branch: refs/heads/master Commit: 3990daaf3b6ca2c5a9f7790030096262efb12cb2 Parents: ed4101d Author: jinxing Authored: Tue Apr 17 08:55:01 2018 -0500 Committer: Imran Rashid Committed: Tue Apr 17 08:55:01 2018 -0500 -- .../apache/spark/scheduler/DAGScheduler.scala | 33 +++-- .../spark/scheduler/DAGSchedulerSuite.scala | 52 2 files changed, 70 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3990daaf/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8c46a84..78b6b34 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1092,17 +1092,16 @@ class DAGScheduler( // the stage as completed here in case there are no tasks to run markStageAsFinished(stage, None) - val debugString = stage match { + stage match { case stage: ShuffleMapStage => - s"Stage ${stage} is actually done; " + -s"(available: ${stage.isAvailable}," + -s"available outputs: ${stage.numAvailableOutputs}," + -s"partitions: ${stage.numPartitions})" + logDebug(s"Stage ${stage} is actually done; " + + s"(available: ${stage.isAvailable}," + + s"available outputs: ${stage.numAvailableOutputs}," + + s"partitions: ${stage.numPartitions})") + markMapStageJobsAsFinished(stage) case stage : ResultStage => - s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" + logDebug(s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})") } - logDebug(debugString) - submitWaitingChildStages(stage) } } @@ -1307,13 +1306,7 @@ class DAGScheduler( shuffleStage.findMissingPartitions().mkString(", ")) submitStage(shuffleStage) } else { -// Mark any map-stage jobs waiting on this stage as finished -if (shuffleStage.mapStageJobs.nonEmpty) { - val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep) - for (job <- shuffleStage.mapStageJobs) { -markMapStageJobAsFinished(job, stats) - } -} +markMapStageJobsAsFinished(shuffleStage) submitWaitingChildStages(shuffleStage) } } @@ -1433,6 +1426,16 @@ class DAGScheduler( } } + private[scheduler] def markMapStageJobsAsFinished(shuffleStage: ShuffleMapStage): Unit = { +// Mark any map-stage jobs waiting on this stage as finished +if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) { + val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep) + for (job <- shuffleStage.mapStageJobs) { +
spark git commit: [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher.
Repository: spark Updated Branches: refs/heads/master 653fe0241 -> 3cb82047f [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher. The current in-process launcher implementation just calls the SparkSubmit object, which, in case of errors, will more often than not exit the JVM. This is not desirable since this launcher is meant to be used inside other applications, and that would kill the application. The change turns SparkSubmit into a class, and abstracts aways some of the functionality used to print error messages and abort the submission process. The default implementation uses the logging system for messages, and throws exceptions for errors. As part of that I also moved some code that doesn't really belong in SparkSubmit to a better location. The command line invocation of spark-submit now uses a special implementation of the SparkSubmit class that overrides those behaviors to do what is expected from the command line version (print to the terminal, exit the JVM, etc). A lot of the changes are to replace calls to methods such as "printErrorAndExit" with the new API. As part of adding tests for this, I had to fix some small things in the launcher option parser so that things like "--version" can work when used in the launcher library. There is still code that prints directly to the terminal, like all the Ivy-related code in SparkSubmitUtils, and other areas where some re-factoring would help, like the CommandLineUtils class, but I chose to leave those alone to keep this change more focused. Aside from existing and added unit tests, I ran command line tools with a bunch of different arguments to make sure messages and errors behave like before. Author: Marcelo VanzinCloses #20925 from vanzin/SPARK-22941. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3cb82047 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3cb82047 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3cb82047 Branch: refs/heads/master Commit: 3cb82047f2f51af553df09b9323796af507d36f8 Parents: 653fe02 Author: Marcelo Vanzin Authored: Wed Apr 11 10:13:44 2018 -0500 Committer: Imran Rashid Committed: Wed Apr 11 10:13:44 2018 -0500 -- .../apache/spark/deploy/DependencyUtils.scala | 30 +- .../org/apache/spark/deploy/SparkSubmit.scala | 318 ++- .../spark/deploy/SparkSubmitArguments.scala | 90 +++--- .../spark/deploy/worker/DriverWrapper.scala | 4 +- .../apache/spark/util/CommandLineUtils.scala| 18 +- .../spark/launcher/SparkLauncherSuite.java | 37 ++- .../apache/spark/deploy/SparkSubmitSuite.scala | 69 ++-- .../deploy/rest/StandaloneRestSubmitSuite.scala | 2 +- .../apache/spark/launcher/AbstractLauncher.java | 6 +- .../spark/launcher/InProcessLauncher.java | 14 +- .../launcher/SparkSubmitCommandBuilder.java | 82 +++-- project/MimaExcludes.scala | 7 +- .../deploy/mesos/MesosClusterDispatcher.scala | 10 +- .../mesos/MesosClusterDispatcherArguments.scala | 6 +- 14 files changed, 401 insertions(+), 292 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3cb82047/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index fac834a..178bdcf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -25,9 +25,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SecurityManager, SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.{MutableURLClassLoader, Utils} -private[deploy] object DependencyUtils { +private[deploy] object DependencyUtils extends Logging { def resolveMavenDependencies( packagesExclusions: String, @@ -75,7 +76,7 @@ private[deploy] object DependencyUtils { def addJarsToClassPath(jars: String, loader: MutableURLClassLoader): Unit = { if (jars != null) { for (jar <- jars.split(",")) { -SparkSubmit.addJarToClasspath(jar, loader) +addJarToClasspath(jar, loader) } } } @@ -151,6 +152,31 @@ private[deploy] object DependencyUtils { }.mkString(",") } + def addJarToClasspath(localJar: String, loader: MutableURLClassLoader): Unit = { +val uri = Utils.resolveURI(localJar) +uri.getScheme match { + case "file" | "local" => +val file = new File(uri.getPath) +if
spark git commit: [SPARK-23053][CORE][BRANCH-2.1] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status
Repository: spark Updated Branches: refs/heads/branch-2.1 1df8020e1 -> 24fe6eb0f [SPARK-23053][CORE][BRANCH-2.1] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status ## What changes were proposed in this pull request? This PR backports [#20244](https://github.com/apache/spark/pull/20244) When we run concurrent jobs using the same rdd which is marked to do checkpoint. If one job has finished running the job, and start the process of RDD.doCheckpoint, while another job is submitted, then submitStage and submitMissingTasks will be called. In [submitMissingTasks](https://github.com/apache/spark/blob/branch-2.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L932), will serialize taskBinaryBytes and calculate task partitions which are both affected by the status of checkpoint, if the former is calculated before doCheckpoint finished, while the latter is calculated after doCheckpoint finished, when run task, rdd.compute will be called, for some rdds with particular partition type such as [UnionRDD](https://github.com/apache/spark/blob/branch-2.1/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala) who will do partition type cast, will get a ClassCastException because the part params is actually a CheckpointRDDPartition. This error occurs because rdd.doCheckpoint occurs in the same thread that called sc.runJob, while the task serialization occurs in the DAGSchedulers event loop. ## How was this patch tested? the exist tests. Author: huangtengfeiCloses #20635 from ivoson/branch-2.1-23053. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/24fe6eb0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/24fe6eb0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/24fe6eb0 Branch: refs/heads/branch-2.1 Commit: 24fe6eb0f6e2fe31d1c93fc65bc294ebb94e2dcf Parents: 1df8020 Author: huangtengfei Authored: Tue Feb 20 21:01:45 2018 -0600 Committer: Imran Rashid Committed: Tue Feb 20 21:01:45 2018 -0600 -- .../apache/spark/scheduler/DAGScheduler.scala | 27 +--- 1 file changed, 18 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/24fe6eb0/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 01a95c0..9d46d69 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -982,15 +982,24 @@ class DAGScheduler( // might modify state of objects referenced in their closures. This is necessary in Hadoop // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null +var partitions: Array[Partition] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = stage match { -case stage: ShuffleMapStage => - JavaUtils.bufferToArray( -closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) -case stage: ResultStage => - JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) + var taskBinaryBytes: Array[Byte] = null + // taskBinaryBytes and partitions are both effected by the checkpoint status. We need + // this synchronization in case another concurrent job is checkpointing this RDD, so we get a + // consistent view of both variables. + RDDCheckpointData.synchronized { +taskBinaryBytes = stage match { + case stage: ShuffleMapStage => +JavaUtils.bufferToArray( + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) + case stage: ResultStage => +JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) +} + +
spark git commit: [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stag…
Repository: spark Updated Branches: refs/heads/branch-2.3 0bd7765cd -> 75bb19a01 [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stag⦠â¦e page ## What changes were proposed in this pull request? Fixing exception got at sorting tasks by Host / Executor ID: ``` java.lang.IllegalArgumentException: Invalid sort column: Host at org.apache.spark.ui.jobs.ApiHelper$.indexName(StagePage.scala:1017) at org.apache.spark.ui.jobs.TaskDataSource.sliceData(StagePage.scala:694) at org.apache.spark.ui.PagedDataSource.pageData(PagedTable.scala:61) at org.apache.spark.ui.PagedTable$class.table(PagedTable.scala:96) at org.apache.spark.ui.jobs.TaskPagedTable.table(StagePage.scala:708) at org.apache.spark.ui.jobs.StagePage.liftedTree1$1(StagePage.scala:293) at org.apache.spark.ui.jobs.StagePage.render(StagePage.scala:282) at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82) at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82) at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90) at javax.servlet.http.HttpServlet.service(HttpServlet.java:687) at javax.servlet.http.HttpServlet.service(HttpServlet.java:790) at org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848) at org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584) ``` Moreover some refactoring to avoid similar problems by introducing constants for each header name and reusing them at the identification of the corresponding sorting index. ## How was this patch tested? Manually: ![screen shot 2018-02-13 at 18 57 10](https://user-images.githubusercontent.com/2017933/36166532-1cfdf3b8-10f3-11e8-8d32-5fcaad2af214.png) (cherry picked from commit 1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea) Author: âattilapirosâCloses #20623 from squito/fix_backport. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/75bb19a0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/75bb19a0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/75bb19a0 Branch: refs/heads/branch-2.3 Commit: 75bb19a018f9260eab3ea0ba3ea46e84b87eabf2 Parents: 0bd7765 Author: âattilapirosâ Authored: Thu Feb 15 14:03:41 2018 -0600 Committer: Imran Rashid Committed: Thu Feb 15 14:03:41 2018 -0600 -- .../org/apache/spark/status/storeTypes.scala| 2 + .../org/apache/spark/ui/jobs/StagePage.scala| 121 --- .../org/apache/spark/ui/StagePageSuite.scala| 63 +- 3 files changed, 139 insertions(+), 47 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/75bb19a0/core/src/main/scala/org/apache/spark/status/storeTypes.scala -- diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 412644d..646cf25 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -109,6 +109,7 @@ private[spark] object TaskIndexNames { final val DURATION = "dur" final val ERROR = "err" final val EXECUTOR = "exe" + final val HOST = "hst" final val EXEC_CPU_TIME = "ect" final val EXEC_RUN_TIME = "ert" final val GC_TIME = "gc" @@ -165,6 +166,7 @@ private[spark] class TaskDataWrapper( val duration: Long, @KVIndexParam(value = TaskIndexNames.EXECUTOR, parent = TaskIndexNames.STAGE) val executorId: String, +@KVIndexParam(value = TaskIndexNames.HOST, parent = TaskIndexNames.STAGE) val host: String, @KVIndexParam(value = TaskIndexNames.STATUS, parent = TaskIndexNames.STAGE) val status: String, http://git-wip-us.apache.org/repos/asf/spark/blob/75bb19a0/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 8af2537..a4710f6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -742,37 +742,39 @@ private[ui] class TaskPagedTable( } def headers: Seq[Node] = { +import ApiHelper._ + val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( -("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), -("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", ""), -("Scheduler Delay",
spark git commit: [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page
Repository: spark Updated Branches: refs/heads/master db45daab9 -> 1dc2c1d5e [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page ## What changes were proposed in this pull request? Fixing exception got at sorting tasks by Host / Executor ID: ``` java.lang.IllegalArgumentException: Invalid sort column: Host at org.apache.spark.ui.jobs.ApiHelper$.indexName(StagePage.scala:1017) at org.apache.spark.ui.jobs.TaskDataSource.sliceData(StagePage.scala:694) at org.apache.spark.ui.PagedDataSource.pageData(PagedTable.scala:61) at org.apache.spark.ui.PagedTable$class.table(PagedTable.scala:96) at org.apache.spark.ui.jobs.TaskPagedTable.table(StagePage.scala:708) at org.apache.spark.ui.jobs.StagePage.liftedTree1$1(StagePage.scala:293) at org.apache.spark.ui.jobs.StagePage.render(StagePage.scala:282) at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82) at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82) at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90) at javax.servlet.http.HttpServlet.service(HttpServlet.java:687) at javax.servlet.http.HttpServlet.service(HttpServlet.java:790) at org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848) at org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584) ``` Moreover some refactoring to avoid similar problems by introducing constants for each header name and reusing them at the identification of the corresponding sorting index. ## How was this patch tested? Manually: ![screen shot 2018-02-13 at 18 57 10](https://user-images.githubusercontent.com/2017933/36166532-1cfdf3b8-10f3-11e8-8d32-5fcaad2af214.png) Author: âattilapirosâCloses #20601 from attilapiros/SPARK-23413. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1dc2c1d5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1dc2c1d5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1dc2c1d5 Branch: refs/heads/master Commit: 1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea Parents: db45daa Author: âattilapirosâ Authored: Thu Feb 15 13:51:24 2018 -0600 Committer: Imran Rashid Committed: Thu Feb 15 13:51:24 2018 -0600 -- .../org/apache/spark/status/storeTypes.scala| 2 + .../org/apache/spark/ui/jobs/StagePage.scala| 121 --- .../org/apache/spark/ui/StagePageSuite.scala| 63 +- 3 files changed, 139 insertions(+), 47 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1dc2c1d5/core/src/main/scala/org/apache/spark/status/storeTypes.scala -- diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 412644d..646cf25 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -109,6 +109,7 @@ private[spark] object TaskIndexNames { final val DURATION = "dur" final val ERROR = "err" final val EXECUTOR = "exe" + final val HOST = "hst" final val EXEC_CPU_TIME = "ect" final val EXEC_RUN_TIME = "ert" final val GC_TIME = "gc" @@ -165,6 +166,7 @@ private[spark] class TaskDataWrapper( val duration: Long, @KVIndexParam(value = TaskIndexNames.EXECUTOR, parent = TaskIndexNames.STAGE) val executorId: String, +@KVIndexParam(value = TaskIndexNames.HOST, parent = TaskIndexNames.STAGE) val host: String, @KVIndexParam(value = TaskIndexNames.STATUS, parent = TaskIndexNames.STAGE) val status: String, http://git-wip-us.apache.org/repos/asf/spark/blob/1dc2c1d5/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 5c2b0c3..a9265d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -750,37 +750,39 @@ private[ui] class TaskPagedTable( } def headers: Seq[Node] = { +import ApiHelper._ + val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( -("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), -("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", ""), -("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), -("Task Deserialization Time",
spark git commit: [SPARK-23235][CORE] Add executor Threaddump to api
Repository: spark Updated Branches: refs/heads/master 2ee76c22b -> a5a4b8350 [SPARK-23235][CORE] Add executor Threaddump to api ## What changes were proposed in this pull request? Extending api with the executor thread dump data. For this new REST URL is introduced: - GET http://localhost:4040/api/v1/applications/{applicationId}/executors/{executorId}/threads Example response: ``` javascript [ { "threadId" : 52, "threadName" : "context-cleaner-periodic-gc", "threadState" : "TIMED_WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1385411893})", "holdingLocks" : [ ] }, { "threadId" : 48, "threadName" : "dag-scheduler-event-loop", "threadState" : "WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)\njava.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)\norg.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:46)", "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1138053349})", "holdingLocks" : [ ] }, { "threadId" : 17, "threadName" : "dispatcher-event-loop-0", "threadState" : "WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})", "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker832743930})" ] }, { "threadId" : 18, "threadName" : "dispatcher-event-loop-1", "threadState" : "WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})", "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker834153999})" ] }, { "threadId" : 19, "threadName" : "dispatcher-event-loop-2", "threadState" : "WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)", "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})", "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker664836465})" ] }, { "threadId" : 20, "threadName" : "dispatcher-event-loop-3", "threadState" : "WAITING", "stackTrace" : "sun.misc.Unsafe.park(Native
spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status
Repository: spark Updated Branches: refs/heads/branch-2.2 14b5dbfa9 -> 73263b215 [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status ## What changes were proposed in this pull request? When we run concurrent jobs using the same rdd which is marked to do checkpoint. If one job has finished running the job, and start the process of RDD.doCheckpoint, while another job is submitted, then submitStage and submitMissingTasks will be called. In [submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961), will serialize taskBinaryBytes and calculate task partitions which are both affected by the status of checkpoint, if the former is calculated before doCheckpoint finished, while the latter is calculated after doCheckpoint finished, when run task, rdd.compute will be called, for some rdds with particular partition type such as [UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala) who will do partition type cast, will get a ClassCastException because the part params is actually a CheckpointRDDPartition. This error occurs because rdd.doCheckpoint occurs in the same thread that called sc.runJob, while the task serialization occurs in the DAGSchedulers event loop. ## How was this patch tested? the exist uts and also add a test case in DAGScheduerSuite to show the exception case. Author: huangtengfeiCloses #20244 from ivoson/branch-taskpart-mistype. (cherry picked from commit 091a000d27f324de8c5c527880854ecfcf5de9a4) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/73263b21 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/73263b21 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/73263b21 Branch: refs/heads/branch-2.2 Commit: 73263b2150ededd8000b3fe4f276625d90de2507 Parents: 14b5dbf Author: huangtengfei Authored: Tue Feb 13 09:59:21 2018 -0600 Committer: Imran Rashid Committed: Tue Feb 13 10:00:23 2018 -0600 -- .../apache/spark/scheduler/DAGScheduler.scala | 27 +--- 1 file changed, 18 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/73263b21/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b009424..87e407f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -992,15 +992,24 @@ class DAGScheduler( // might modify state of objects referenced in their closures. This is necessary in Hadoop // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null +var partitions: Array[Partition] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = stage match { -case stage: ShuffleMapStage => - JavaUtils.bufferToArray( -closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) -case stage: ResultStage => - JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) + var taskBinaryBytes: Array[Byte] = null + // taskBinaryBytes and partitions are both effected by the checkpoint status. We need + // this synchronization in case another concurrent job is checkpointing this RDD, so we get a + // consistent view of both variables. + RDDCheckpointData.synchronized { +taskBinaryBytes = stage match { + case stage: ShuffleMapStage => +JavaUtils.bufferToArray( + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) + case stage: ResultStage => +
spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status
Repository: spark Updated Branches: refs/heads/branch-2.3 1c81c0c62 -> dbb1b399b [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status ## What changes were proposed in this pull request? When we run concurrent jobs using the same rdd which is marked to do checkpoint. If one job has finished running the job, and start the process of RDD.doCheckpoint, while another job is submitted, then submitStage and submitMissingTasks will be called. In [submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961), will serialize taskBinaryBytes and calculate task partitions which are both affected by the status of checkpoint, if the former is calculated before doCheckpoint finished, while the latter is calculated after doCheckpoint finished, when run task, rdd.compute will be called, for some rdds with particular partition type such as [UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala) who will do partition type cast, will get a ClassCastException because the part params is actually a CheckpointRDDPartition. This error occurs because rdd.doCheckpoint occurs in the same thread that called sc.runJob, while the task serialization occurs in the DAGSchedulers event loop. ## How was this patch tested? the exist uts and also add a test case in DAGScheduerSuite to show the exception case. Author: huangtengfeiCloses #20244 from ivoson/branch-taskpart-mistype. (cherry picked from commit 091a000d27f324de8c5c527880854ecfcf5de9a4) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dbb1b399 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dbb1b399 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dbb1b399 Branch: refs/heads/branch-2.3 Commit: dbb1b399b6cf8372a3659c472f380142146b1248 Parents: 1c81c0c Author: huangtengfei Authored: Tue Feb 13 09:59:21 2018 -0600 Committer: Imran Rashid Committed: Tue Feb 13 10:00:05 2018 -0600 -- .../apache/spark/scheduler/DAGScheduler.scala | 27 +--- 1 file changed, 18 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dbb1b399/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 199937b..8c46a84 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1016,15 +1016,24 @@ class DAGScheduler( // might modify state of objects referenced in their closures. This is necessary in Hadoop // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null +var partitions: Array[Partition] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = stage match { -case stage: ShuffleMapStage => - JavaUtils.bufferToArray( -closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) -case stage: ResultStage => - JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) + var taskBinaryBytes: Array[Byte] = null + // taskBinaryBytes and partitions are both effected by the checkpoint status. We need + // this synchronization in case another concurrent job is checkpointing this RDD, so we get a + // consistent view of both variables. + RDDCheckpointData.synchronized { +taskBinaryBytes = stage match { + case stage: ShuffleMapStage => +JavaUtils.bufferToArray( + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) + case stage: ResultStage => +
spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status
Repository: spark Updated Branches: refs/heads/master d6e1958a2 -> 091a000d2 [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status ## What changes were proposed in this pull request? When we run concurrent jobs using the same rdd which is marked to do checkpoint. If one job has finished running the job, and start the process of RDD.doCheckpoint, while another job is submitted, then submitStage and submitMissingTasks will be called. In [submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961), will serialize taskBinaryBytes and calculate task partitions which are both affected by the status of checkpoint, if the former is calculated before doCheckpoint finished, while the latter is calculated after doCheckpoint finished, when run task, rdd.compute will be called, for some rdds with particular partition type such as [UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala) who will do partition type cast, will get a ClassCastException because the part params is actually a CheckpointRDDPartition. This error occurs because rdd.doCheckpoint occurs in the same thread that called sc.runJob, while the task serialization occurs in the DAGSchedulers event loop. ## How was this patch tested? the exist uts and also add a test case in DAGScheduerSuite to show the exception case. Author: huangtengfeiCloses #20244 from ivoson/branch-taskpart-mistype. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/091a000d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/091a000d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/091a000d Branch: refs/heads/master Commit: 091a000d27f324de8c5c527880854ecfcf5de9a4 Parents: d6e1958 Author: huangtengfei Authored: Tue Feb 13 09:59:21 2018 -0600 Committer: Imran Rashid Committed: Tue Feb 13 09:59:21 2018 -0600 -- .../apache/spark/scheduler/DAGScheduler.scala | 27 +--- 1 file changed, 18 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/091a000d/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 199937b..8c46a84 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1016,15 +1016,24 @@ class DAGScheduler( // might modify state of objects referenced in their closures. This is necessary in Hadoop // where the JobConf/Configuration object is not thread-safe. var taskBinary: Broadcast[Array[Byte]] = null +var partitions: Array[Partition] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = stage match { -case stage: ShuffleMapStage => - JavaUtils.bufferToArray( -closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) -case stage: ResultStage => - JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) + var taskBinaryBytes: Array[Byte] = null + // taskBinaryBytes and partitions are both effected by the checkpoint status. We need + // this synchronization in case another concurrent job is checkpointing this RDD, so we get a + // consistent view of both variables. + RDDCheckpointData.synchronized { +taskBinaryBytes = stage match { + case stage: ShuffleMapStage => +JavaUtils.bufferToArray( + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) + case stage: ResultStage => +JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) +} + +partitions =
spark git commit: [SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab
Repository: spark Updated Branches: refs/heads/master 116c581d2 -> d6e1958a2 [SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab ## What changes were proposed in this pull request? The purpose of this PR to reflect the stage level blacklisting on the executor tab for the currently active stages. After this change in the executor tab at the Status column one of the following label will be: - "Blacklisted" when the executor is blacklisted application level (old flag) - "Dead" when the executor is not Blacklisted and not Active - "Blacklisted in Stages: [...]" when the executor is Active but the there are active blacklisted stages for the executor. Within the [] coma separated active stageIDs are listed. - "Active" when the executor is Active and there is no active blacklisted stages for the executor ## How was this patch tested? Both with unit tests and manually. Manual test Spark was started as: ```bash bin/spark-shell --master "local-cluster[2,1,1024]" --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10" ``` And the job was: ```scala import org.apache.spark.SparkEnv val pairs = sc.parallelize(1 to 1, 10).map { x => if (SparkEnv.get.executorId.toInt == 0) throw new RuntimeException("Bad executor") else { Thread.sleep(10) (x % 10, x) } } val all = pairs.cogroup(pairs) all.collect() ``` UI screenshots about the running: - One executor is blacklisted in the two stages: ![One executor is blacklisted in two stages](https://issues.apache.org/jira/secure/attachment/12908314/multiple_stages_1.png) - One stage completes the other one is still running: ![One stage completes the other is still running](https://issues.apache.org/jira/secure/attachment/12908315/multiple_stages_2.png) - Both stages are completed: ![Both stages are completed](https://issues.apache.org/jira/secure/attachment/12908316/multiple_stages_3.png) ### Unit tests In AppStatusListenerSuite.scala both the node blacklisting for a stage and the executor blacklisting for stage are tested. Author: âattilapirosâCloses #20408 from attilapiros/SPARK-23189. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d6e1958a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d6e1958a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d6e1958a Branch: refs/heads/master Commit: d6e1958a2472898e60bd013902c2f35111596e40 Parents: 116c581 Author: âattilapirosâ Authored: Tue Feb 13 09:54:52 2018 -0600 Committer: Imran Rashid Committed: Tue Feb 13 09:54:52 2018 -0600 -- .../org/apache/spark/ui/static/executorspage.js | 21 ++--- .../apache/spark/status/AppStatusListener.scala | 49 +++- .../org/apache/spark/status/LiveEntity.scala| 7 ++- .../org/apache/spark/status/api/v1/api.scala| 3 +- .../executor_list_json_expectation.json | 3 +- .../executor_memory_usage_expectation.json | 15 -- .../executor_node_blacklisting_expectation.json | 15 -- ...blacklisting_unblacklisting_expectation.json | 15 -- .../spark/status/AppStatusListenerSuite.scala | 21 + 9 files changed, 113 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d6e1958a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index d430d8c..6717af3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -25,12 +25,18 @@ function getThreadDumpEnabled() { return threadDumpEnabled; } -function formatStatus(status, type) { +function formatStatus(status, type, row) { +if (row.isBlacklisted) { +return "Blacklisted"; +} + if (status) { -return "Active" -} else { -return "Dead" +if (row.blacklistedInStages.length == 0) { +return "Active" +} +return "Active (Blacklisted in Stages: [" + row.blacklistedInStages.join(", ") + "])"; } +return "Dead" } jQuery.extend(jQuery.fn.dataTableExt.oSort, { @@ -415,9 +421,10 @@ $(document).ready(function () { } }, {data: 'hostPort'}, -{data: 'isActive', render: function (data, type, row) { -if (row.isBlacklisted) return
spark git commit: [SPARK-23253][CORE][SHUFFLE] Only write shuffle temporary index file when there is not an existing one
Repository: spark Updated Branches: refs/heads/master b9503fcbb -> dd52681bf [SPARK-23253][CORE][SHUFFLE] Only write shuffle temporary index file when there is not an existing one ## What changes were proposed in this pull request? Shuffle Index temporay file is used for atomic creating shuffle index file, it is not needed when the index file already exists after another attempts of same task had it done. ## How was this patch tested? exitsting ut cc squito Author: Kent YaoCloses #20422 from yaooqinn/SPARK-23253. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dd52681b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dd52681b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dd52681b Branch: refs/heads/master Commit: dd52681bf542386711609cb037a55b3d264eddef Parents: b9503fc Author: Kent Yao Authored: Fri Feb 2 09:10:50 2018 -0600 Committer: Imran Rashid Committed: Fri Feb 2 09:10:50 2018 -0600 -- .../shuffle/IndexShuffleBlockResolver.scala | 27 + .../sort/IndexShuffleBlockResolverSuite.scala | 59 ++-- 2 files changed, 56 insertions(+), 30 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dd52681b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala -- diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 266ee42..c5f3f6e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -141,19 +141,6 @@ private[spark] class IndexShuffleBlockResolver( val indexFile = getIndexFile(shuffleId, mapId) val indexTmp = Utils.tempFileWith(indexFile) try { - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) - Utils.tryWithSafeFinally { -// We take in lengths of each block, need to convert it to offsets. -var offset = 0L -out.writeLong(offset) -for (length <- lengths) { - offset += length - out.writeLong(offset) -} - } { -out.close() - } - val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. @@ -166,10 +153,22 @@ private[spark] class IndexShuffleBlockResolver( if (dataTmp != null && dataTmp.exists()) { dataTmp.delete() } - indexTmp.delete() } else { // This is the first successful attempt in writing the map outputs for this task, // so override any existing index and data files with the ones we wrote. + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) + Utils.tryWithSafeFinally { +// We take in lengths of each block, need to convert it to offsets. +var offset = 0L +out.writeLong(offset) +for (length <- lengths) { + offset += length + out.writeLong(offset) +} + } { +out.close() + } + if (indexFile.exists()) { indexFile.delete() } http://git-wip-us.apache.org/repos/asf/spark/blob/dd52681b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index d21ce73..4ce379b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle.sort -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{DataInputStream, File, FileInputStream, FileOutputStream} import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS @@ -64,6 +64,9 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } test("commit shuffle files multiple times") { +val shuffleId = 1 +val mapId = 2 +val idxName = s"shuffle_${shuffleId}_${mapId}_0.index" val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths =
spark git commit: [SPARK-23209][core] Allow credential manager to work when Hive not available.
Repository: spark Updated Branches: refs/heads/branch-2.3 438631031 -> 75131ee86 [SPARK-23209][core] Allow credential manager to work when Hive not available. The JVM seems to be doing early binding of classes that the Hive provider depends on, causing an error to be thrown before it was caught by the code in the class. The fix wraps the creation of the provider in a try..catch so that the provider can be ignored when dependencies are missing. Added a unit test (which fails without the fix), and also tested that getting tokens still works in a real cluster. Author: Marcelo VanzinCloses #20399 from vanzin/SPARK-23209. (cherry picked from commit b834446ec1338349f6d974afd96f677db3e8fd1a) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/75131ee8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/75131ee8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/75131ee8 Branch: refs/heads/branch-2.3 Commit: 75131ee867bca17ca3aade5f832f1d49b7cfcff5 Parents: 4386310 Author: Marcelo Vanzin Authored: Mon Jan 29 16:09:14 2018 -0600 Committer: Imran Rashid Committed: Mon Jan 29 16:09:28 2018 -0600 -- .../security/HadoopDelegationTokenManager.scala | 17 +- .../HadoopDelegationTokenManagerSuite.scala | 58 2 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/75131ee8/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 116a686..5151df0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -64,9 +64,9 @@ private[spark] class HadoopDelegationTokenManager( } private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { -val providers = List(new HadoopFSDelegationTokenProvider(fileSystems), - new HiveDelegationTokenProvider, - new HBaseDelegationTokenProvider) +val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++ + safeCreateProvider(new HiveDelegationTokenProvider) ++ + safeCreateProvider(new HBaseDelegationTokenProvider) // Filter out providers for which spark.security.credentials.{service}.enabled is false. providers @@ -75,6 +75,17 @@ private[spark] class HadoopDelegationTokenManager( .toMap } + private def safeCreateProvider( + createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { +try { + Some(createFn) +} catch { + case t: Throwable => +logDebug(s"Failed to load built in provider.", t) +None +} + } + def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) http://git-wip-us.apache.org/repos/asf/spark/blob/75131ee8/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index eeffc36..2849a10 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.security +import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials @@ -110,7 +111,64 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { creds.getAllTokens.size should be (0) } + test("SPARK-23209: obtain tokens when Hive classes are not available") { +// This test needs a custom class loader to hide Hive classes which are in the classpath. +// Because the manager code loads the Hive provider directly instead of using reflection, we +// need to drive the test through the custom class loader so a new copy that cannot find +// Hive classes is loaded. +val currentLoader = Thread.currentThread().getContextClassLoader() +val noHive = new ClassLoader() { + override def loadClass(name:
spark git commit: [SPARK-23209][core] Allow credential manager to work when Hive not available.
Repository: spark Updated Branches: refs/heads/master e30b34f7b -> b834446ec [SPARK-23209][core] Allow credential manager to work when Hive not available. The JVM seems to be doing early binding of classes that the Hive provider depends on, causing an error to be thrown before it was caught by the code in the class. The fix wraps the creation of the provider in a try..catch so that the provider can be ignored when dependencies are missing. Added a unit test (which fails without the fix), and also tested that getting tokens still works in a real cluster. Author: Marcelo VanzinCloses #20399 from vanzin/SPARK-23209. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b834446e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b834446e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b834446e Branch: refs/heads/master Commit: b834446ec1338349f6d974afd96f677db3e8fd1a Parents: e30b34f Author: Marcelo Vanzin Authored: Mon Jan 29 16:09:14 2018 -0600 Committer: Imran Rashid Committed: Mon Jan 29 16:09:14 2018 -0600 -- .../security/HadoopDelegationTokenManager.scala | 17 +- .../HadoopDelegationTokenManagerSuite.scala | 58 2 files changed, 72 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b834446e/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 116a686..5151df0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -64,9 +64,9 @@ private[spark] class HadoopDelegationTokenManager( } private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { -val providers = List(new HadoopFSDelegationTokenProvider(fileSystems), - new HiveDelegationTokenProvider, - new HBaseDelegationTokenProvider) +val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++ + safeCreateProvider(new HiveDelegationTokenProvider) ++ + safeCreateProvider(new HBaseDelegationTokenProvider) // Filter out providers for which spark.security.credentials.{service}.enabled is false. providers @@ -75,6 +75,17 @@ private[spark] class HadoopDelegationTokenManager( .toMap } + private def safeCreateProvider( + createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { +try { + Some(createFn) +} catch { + case t: Throwable => +logDebug(s"Failed to load built in provider.", t) +None +} + } + def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) http://git-wip-us.apache.org/repos/asf/spark/blob/b834446e/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index eeffc36..2849a10 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.security +import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials @@ -110,7 +111,64 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { creds.getAllTokens.size should be (0) } + test("SPARK-23209: obtain tokens when Hive classes are not available") { +// This test needs a custom class loader to hide Hive classes which are in the classpath. +// Because the manager code loads the Hive provider directly instead of using reflection, we +// need to drive the test through the custom class loader so a new copy that cannot find +// Hive classes is loaded. +val currentLoader = Thread.currentThread().getContextClassLoader() +val noHive = new ClassLoader() { + override def loadClass(name: String, resolve: Boolean): Class[_] = { +if (name.startsWith("org.apache.hive") ||
[2/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/spark-events/application_1516285256255_0012 -- diff --git a/core/src/test/resources/spark-events/application_1516285256255_0012 b/core/src/test/resources/spark-events/application_1516285256255_0012 new file mode 100755 index 000..3e1736c --- /dev/null +++ b/core/src/test/resources/spark-events/application_1516285256255_0012 @@ -0,0 +1,71 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.3.0-SNAPSHOT"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.enabled":"true","spark.driver.host":"apiros-1.gce.test.com","spark.eventLog.enabled":"true","spark.driver.port":"33058","spark.repl.class.uri":"spark://apiros-1.gce.test.com:33058/classes","spark.jars":"","spark.repl.class.outputDir":"/tmp/spark-6781fb17-e07a-4b32-848b-9936c2e88b33/repl-c0fd7008-04be-471e-a173-6ad3e62d53d7","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"1","spark.scheduler.mode":"FIFO","spark.executor.instances":"8","spark.ui.showConsoleProgress":"true","spark.blacklist.stage.maxFailedTasksPerExecutor":"1","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.e xecutor.memory":"2G","spark.home":"/github/spark","spark.sql.catalogImplementation":"hive","spark.driver.appUIAddress":"http://apiros-1.gce.test.com:4040","spark.blacklist.application.maxFailedTasksPerExecutor":"10","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"apiros-1.gce.test.com","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://apiros-1.gce.test.com:8088/proxy/application_1516285256255_0012","spark.app.id":"application_1516285256255_0012"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","su n.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/amd64","user.dir":"*(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1. 8.0.161-0.b14.el7_4.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"*(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-693.5.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*(redacted)","java.vendor.url":"*(redacted)","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.nam e":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"*(redacted)","user.name":"*(redacted)","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --conf spark.blacklist.stage.maxFailedTasksPerExecutor=1 --conf spark.blacklist.enabled=true --conf
[4/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json -- diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 31093a6..03f886a 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -421,7 +421,8 @@ "shuffleWrite" : 13180, "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false } }, "killedTasksSummary" : { } http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json -- diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 601d706..947c899 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -421,7 +421,8 @@ "shuffleWrite" : 13180, "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false } }, "killedTasksSummary" : { } http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json -- diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 9cdcef0..963f010 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -465,7 +465,8 @@ "shuffleWrite" : 0, "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false } }, "killedTasksSummary" : { } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[1/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
Repository: spark Updated Branches: refs/heads/master de36f65d3 -> 0ec95bb7d http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 7aa60f2..87f12f3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -156,6 +156,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "applications/local-1426533911241/1/stages/0/0/taskList", "stage task list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/stages/0/0/taskList", +"blacklisting for stage" -> "applications/app-20180109111548-/stages/0/0", +"blacklisting node for stage" -> "applications/application_1516285256255_0012/stages/0/0", "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", "executor node blacklisting" -> "applications/app-20161116163331-/executors", http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index cd1b7a9..afebcdd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -92,7 +92,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = { -new TaskSetBlacklist(conf, stageId, clock) +new TaskSetBlacklist(listenerBusMock, conf, stageId, stageAttemptId = 0, clock = clock) } test("executors can be blacklisted with only a few failures per stage") { http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index 18981d5..6e2709d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -16,18 +16,32 @@ */ package org.apache.spark.scheduler +import org.mockito.Matchers.isA +import org.mockito.Mockito.{never, verify} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.mockito.MockitoSugar + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config -import org.apache.spark.util.{ManualClock, SystemClock} +import org.apache.spark.util.ManualClock + +class TaskSetBlacklistSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { -class TaskSetBlacklistSuite extends SparkFunSuite { + private var listenerBusMock: LiveListenerBus = _ + + override def beforeEach(): Unit = { +listenerBusMock = mock[LiveListenerBus] +super.beforeEach() + } test("Blacklisting tasks, executors, and nodes") { val conf = new SparkConf().setAppName("test").setMaster("local") .set(config.BLACKLIST_ENABLED.key, "true") val clock = new ManualClock +val attemptId = 0 +val taskSetBlacklist = new TaskSetBlacklist( + listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) -val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock) clock.setTime(0) // We will mark task 0 & 1 failed on both executor 1 & 2. // We should blacklist all executors on that host, for all tasks for the stage. Note the API @@ -46,27 +60,53 @@ class TaskSetBlacklistSuite extends SparkFunSuite { val shouldBeBlacklisted = (executor == "exec1" && index == 0) assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted) } + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) +verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerExecutorBlacklistedForStage])) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) +verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) // Mark task 1 failed on exec1 -- this pushes the executor into the blacklist taskSetBlacklist.updateBlacklistForFailedTask( "hostA", exec = "exec1", index = 1,
[5/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
}} -{executor.map(_.isBlacklisted).getOrElse(false)} +{ + if (executor.map(_.isBlacklisted).getOrElse(false)) { +for application + } else if (v.isBlacklistedForStage) { +for stage + } else { +false + } +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json -- diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index f2c3ec5..4fecf84 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,34 @@ [ { + "id" : "application_1516285256255_0012", + "name" : "Spark shell", + "attempts" : [ { +"startTime" : "2018-01-18T18:30:35.119GMT", +"endTime" : "2018-01-18T18:38:27.938GMT", +"lastUpdated" : "", +"duration" : 472819, +"sparkUser" : "attilapiros", +"completed" : true, +"appSparkVersion" : "2.3.0-SNAPSHOT", +"lastUpdatedEpoch" : 0, +"startTimeEpoch" : 1516300235119, +"endTimeEpoch" : 1516300707938 + } ] +}, { + "id" : "app-20180109111548-", + "name" : "Spark shell", + "attempts" : [ { +"startTime" : "2018-01-09T10:15:42.372GMT", +"endTime" : "2018-01-09T10:24:37.606GMT", +"lastUpdated" : "", +"duration" : 535234, +"sparkUser" : "attilapiros", +"completed" : true, +"appSparkVersion" : "2.3.0-SNAPSHOT", +"lastUpdatedEpoch" : 0, +"startTimeEpoch" : 1515492942372, +"endTimeEpoch" : 1515493477606 + } ] +}, { "id" : "app-20161116163331-", "name" : "Spark shell", "attempts" : [ { @@ -9,9 +39,9 @@ "sparkUser" : "jose", "completed" : true, "appSparkVersion" : "2.1.0-SNAPSHOT", -"endTimeEpoch" : 1479335620587, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1479335609916, - "lastUpdatedEpoch" : 0 +"endTimeEpoch" : 1479335620587 } ] }, { "id" : "app-20161115172038-", @@ -24,9 +54,9 @@ "sparkUser" : "jose", "completed" : true, "appSparkVersion" : "2.1.0-SNAPSHOT", -"endTimeEpoch" : 1479252138874, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1479252037079, -"lastUpdatedEpoch" : 0 +"endTimeEpoch" : 1479252138874 } ] }, { "id" : "local-1430917381534", @@ -39,9 +69,9 @@ "sparkUser" : "irashid", "completed" : true, "appSparkVersion" : "1.4.0-SNAPSHOT", -"endTimeEpoch" : 1430917391398, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1430917380893, -"lastUpdatedEpoch" : 0 +"endTimeEpoch" : 1430917391398 } ] }, { "id" : "local-1430917381535", @@ -55,9 +85,9 @@ "sparkUser" : "irashid", "completed" : true, "appSparkVersion" : "1.4.0-SNAPSHOT", -"endTimeEpoch" : 1430917380950, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1430917380893, -"lastUpdatedEpoch" : 0 +"endTimeEpoch" : 1430917380950 }, { "attemptId" : "1", "startTime" : "2015-05-06T13:03:00.880GMT", @@ -67,9 +97,9 @@ "sparkUser" : "irashid", "completed" : true, "appSparkVersion" : "1.4.0-SNAPSHOT", -"endTimeEpoch" : 1430917380890, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1430917380880, -"lastUpdatedEpoch" : 0 +"endTimeEpoch" : 1430917380890 } ] }, { "id" : "local-1426533911241", @@ -83,9 +113,9 @@ "sparkUser" : "irashid", "completed" : true, "appSparkVersion" : "", -"endTimeEpoch" : 1426633945177, +"lastUpdatedEpoch" : 0, "startTimeEpoch" : 1426633910242, -"lastUpdated
[3/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/spark-events/app-20180109111548- -- diff --git a/core/src/test/resources/spark-events/app-20180109111548- b/core/src/test/resources/spark-events/app-20180109111548- new file mode 100755 index 000..50893d3 --- /dev/null +++ b/core/src/test/resources/spark-events/app-20180109111548- @@ -0,0 +1,59 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.3.0-SNAPSHOT"} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre","Java Version":"1.8.0_152 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.enabled":"true","spark.driver.host":"172.30.65.138","spark.eventLog.enabled":"true","spark.driver.port":"64273","spark.repl.class.uri":"spark://172.30.65.138:64273/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/9g/gf583nd1765cvfgb_lsvwgp0gp/T/spark-811c1b49-eb66-4bfb-91ae-33b45efa269d/repl-c4438f51-ee23-41ed-8e04-71496e2f40f5","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.ui.showConsoleProgress":"true","spark.blacklist.stage.maxFailedTasksPerExecutor":"1","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"local-cluster[2,1,1024]","spark.home":"*(redacted)","spark.sql.catalogImplementation":"in-memory","spark.blacklist.application.maxF ailedTasksPerExecutor":"10","spark.app.id":"app-20180109111548-"},"System Properties":{"java.io.tmpdir":"/var/folders/9g/gf583nd1765cvfgb_lsvwgp0gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib","user.dir":"*(redacted)","java.library.path":"*(redacted)","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.152-b16","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/l ib/endorsed","java.runtime.version":"1.8.0_152-b16","java.vm.info":"mixed mode","java.ext.dirs":"*(redacted)","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/classes","file .encoding":"UTF-8","user.timezone":"*(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.12.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"*(redacted)","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"*(redacted)","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"*(redacted)","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"*(redacted)","user.name":"*(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[2,1,1024] --conf spark.blacklist.stage.maxF ailedTasksPerExecutor=1 --conf spark.blacklist.enabled=true --conf spark.blacklist.application.maxFailedTasksPerExecutor=10 --conf spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark shell spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre","java.version":"1.8.0_152","sun.io.unicode.encoding":"UnicodeBig"},"Classpath
spark git commit: [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.
Repository: spark Updated Branches: refs/heads/branch-2.3 4b79514c9 -> d0cb19873 [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB. The code was sorting "0" as "less than" negative values, which is a little wrong. Fix is simple, most of the changes are the added test and related cleanup. Author: Marcelo VanzinCloses #20284 from vanzin/SPARK-23103. (cherry picked from commit aa3a1276f9e23ffbb093d00743e63cd4369f9f57) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0cb1987 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0cb1987 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0cb1987 Branch: refs/heads/branch-2.3 Commit: d0cb19873bb325be7e31de62b0ba117dd6b92619 Parents: 4b79514 Author: Marcelo Vanzin Authored: Fri Jan 19 13:32:20 2018 -0600 Committer: Imran Rashid Committed: Fri Jan 19 13:32:38 2018 -0600 -- .../spark/util/kvstore/LevelDBTypeInfo.java | 2 +- .../spark/util/kvstore/DBIteratorSuite.java | 7 +- .../apache/spark/util/kvstore/LevelDBSuite.java | 77 ++-- .../spark/status/AppStatusListenerSuite.scala | 8 +- 4 files changed, 52 insertions(+), 42 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java -- diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java index 232ee41..f4d3592 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java @@ -493,7 +493,7 @@ class LevelDBTypeInfo { byte[] key = new byte[bytes * 2 + 2]; long longValue = ((Number) value).longValue(); key[0] = prefix; -key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; +key[1] = longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; for (int i = 0; i < key.length - 2; i++) { int masked = (int) ((longValue >>> (4 * i)) & 0xF); http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java -- diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java index 9a81f86..1e06243 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java @@ -73,7 +73,9 @@ public abstract class DBIteratorSuite { private static final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); - private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; + private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> { +return Integer.valueOf(t1.num).compareTo(t2.num); + }; private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); /** @@ -112,7 +114,8 @@ public abstract class DBIteratorSuite { t.key = "key" + i; t.id = "id" + i; t.name = "name" + RND.nextInt(MAX_ENTRIES); - t.num = RND.nextInt(MAX_ENTRIES); + // Force one item to have an integer value of zero to test the fix for SPARK-23103. + t.num = (i != 0) ? (int) RND.nextLong() : 0; t.child = "child" + (i % MIN_ENTRIES); allEntries.add(t); } http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java -- diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 2b07d24..b8123ac 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -21,6 +21,8 @@ import java.io.File; import java.util.Arrays; import java.util.List; import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import
spark git commit: [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.
Repository: spark Updated Branches: refs/heads/master fed2139f0 -> aa3a1276f [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB. The code was sorting "0" as "less than" negative values, which is a little wrong. Fix is simple, most of the changes are the added test and related cleanup. Author: Marcelo VanzinCloses #20284 from vanzin/SPARK-23103. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aa3a1276 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aa3a1276 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aa3a1276 Branch: refs/heads/master Commit: aa3a1276f9e23ffbb093d00743e63cd4369f9f57 Parents: fed2139 Author: Marcelo Vanzin Authored: Fri Jan 19 13:32:20 2018 -0600 Committer: Imran Rashid Committed: Fri Jan 19 13:32:20 2018 -0600 -- .../spark/util/kvstore/LevelDBTypeInfo.java | 2 +- .../spark/util/kvstore/DBIteratorSuite.java | 7 +- .../apache/spark/util/kvstore/LevelDBSuite.java | 77 ++-- .../spark/status/AppStatusListenerSuite.scala | 8 +- 4 files changed, 52 insertions(+), 42 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java -- diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java index 232ee41..f4d3592 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java @@ -493,7 +493,7 @@ class LevelDBTypeInfo { byte[] key = new byte[bytes * 2 + 2]; long longValue = ((Number) value).longValue(); key[0] = prefix; -key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; +key[1] = longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; for (int i = 0; i < key.length - 2; i++) { int masked = (int) ((longValue >>> (4 * i)) & 0xF); http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java -- diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java index 9a81f86..1e06243 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java @@ -73,7 +73,9 @@ public abstract class DBIteratorSuite { private static final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); - private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; + private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> { +return Integer.valueOf(t1.num).compareTo(t2.num); + }; private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); /** @@ -112,7 +114,8 @@ public abstract class DBIteratorSuite { t.key = "key" + i; t.id = "id" + i; t.name = "name" + RND.nextInt(MAX_ENTRIES); - t.num = RND.nextInt(MAX_ENTRIES); + // Force one item to have an integer value of zero to test the fix for SPARK-23103. + t.num = (i != 0) ? (int) RND.nextLong() : 0; t.child = "child" + (i % MIN_ENTRIES); allEntries.add(t); } http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java -- diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 2b07d24..b8123ac 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -21,6 +21,8 @@ import java.io.File; import java.util.Arrays; import java.util.List; import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import org.apache.commons.io.FileUtils; import org.iq80.leveldb.DBIterator; @@ -74,11 +76,7 @@ public class
spark git commit: [SPARK-20664][CORE] Delete stale application data from SHS.
Repository: spark Updated Branches: refs/heads/branch-2.3 ffe45913d -> 4b79514c9 [SPARK-20664][CORE] Delete stale application data from SHS. Detect the deletion of event log files from storage, and remove data about the related application attempt in the SHS. Also contains code to fix SPARK-21571 based on code by ericvandenbergfb. Author: Marcelo VanzinCloses #20138 from vanzin/SPARK-20664. (cherry picked from commit fed2139f053fac4a9a6952ff0ab1cc2a5f657bd0) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4b79514c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4b79514c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4b79514c Branch: refs/heads/branch-2.3 Commit: 4b79514c90ca76674d17fd80d125e9dbfb0e845e Parents: ffe4591 Author: Marcelo Vanzin Authored: Fri Jan 19 13:26:37 2018 -0600 Committer: Imran Rashid Committed: Fri Jan 19 13:27:02 2018 -0600 -- .../deploy/history/FsHistoryProvider.scala | 297 --- .../deploy/history/FsHistoryProviderSuite.scala | 117 +++- .../deploy/history/HistoryServerSuite.scala | 4 +- 3 files changed, 306 insertions(+), 112 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4b79514c/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 94c80eb..f9d0b5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} import java.util.{Date, ServiceLoader, UUID} -import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} +import java.util.concurrent.{ExecutorService, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConverters._ @@ -29,7 +29,7 @@ import scala.xml.Node import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams -import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} +import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hdfs.DistributedFileSystem @@ -116,8 +116,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // Used by check event thread and clean log thread. // Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs // and applications between check task and clean task. - private val pool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder() -.setNameFormat("spark-history-task-%d").setDaemon(true).build()) + private val pool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-history-task-%d") // The modification time of the newest log detected during the last scan. Currently only // used for logging msgs (logs are re-scanned based on file size, rather than modtime) @@ -174,7 +173,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Fixed size thread pool to fetch and parse log files. */ private val replayExecutor: ExecutorService = { -if (!conf.contains("spark.testing")) { +if (Utils.isTesting) { ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, "log-replay-executor") } else { MoreExecutors.sameThreadExecutor() @@ -275,7 +274,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { Some(load(appId).toApplicationInfo()) } catch { - case e: NoSuchElementException => + case _: NoSuchElementException => None } } @@ -405,49 +404,70 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { val newLastScanTime = getNewLastScanTime() logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") - // scan for modified applications, replay and merge them - val logInfos = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) + + val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) .filter { entry => !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would
spark git commit: [SPARK-20664][CORE] Delete stale application data from SHS.
Repository: spark Updated Branches: refs/heads/master 07296a61c -> fed2139f0 [SPARK-20664][CORE] Delete stale application data from SHS. Detect the deletion of event log files from storage, and remove data about the related application attempt in the SHS. Also contains code to fix SPARK-21571 based on code by ericvandenbergfb. Author: Marcelo VanzinCloses #20138 from vanzin/SPARK-20664. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fed2139f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fed2139f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fed2139f Branch: refs/heads/master Commit: fed2139f053fac4a9a6952ff0ab1cc2a5f657bd0 Parents: 07296a6 Author: Marcelo Vanzin Authored: Fri Jan 19 13:26:37 2018 -0600 Committer: Imran Rashid Committed: Fri Jan 19 13:26:37 2018 -0600 -- .../deploy/history/FsHistoryProvider.scala | 297 --- .../deploy/history/FsHistoryProviderSuite.scala | 117 +++- .../deploy/history/HistoryServerSuite.scala | 4 +- 3 files changed, 306 insertions(+), 112 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fed2139f/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 94c80eb..f9d0b5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} import java.util.{Date, ServiceLoader, UUID} -import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit} +import java.util.concurrent.{ExecutorService, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConverters._ @@ -29,7 +29,7 @@ import scala.xml.Node import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams -import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} +import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.hdfs.DistributedFileSystem @@ -116,8 +116,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // Used by check event thread and clean log thread. // Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs // and applications between check task and clean task. - private val pool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder() -.setNameFormat("spark-history-task-%d").setDaemon(true).build()) + private val pool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-history-task-%d") // The modification time of the newest log detected during the last scan. Currently only // used for logging msgs (logs are re-scanned based on file size, rather than modtime) @@ -174,7 +173,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Fixed size thread pool to fetch and parse log files. */ private val replayExecutor: ExecutorService = { -if (!conf.contains("spark.testing")) { +if (Utils.isTesting) { ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, "log-replay-executor") } else { MoreExecutors.sameThreadExecutor() @@ -275,7 +274,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { Some(load(appId).toApplicationInfo()) } catch { - case e: NoSuchElementException => + case _: NoSuchElementException => None } } @@ -405,49 +404,70 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { val newLastScanTime = getNewLastScanTime() logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") - // scan for modified applications, replay and merge them - val logInfos = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) + + val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) .filter { entry => !entry.isDirectory() && // FsHistoryProvider generates a hidden file which can't be read. Accidentally // reading a garbage file is safe, but we would log an error which can be scary to // the end-user. !entry.getPath().getName().startsWith(".") && -
spark git commit: [SPARK-22850][CORE] Ensure queued events are delivered to all event queues.
Repository: spark Updated Branches: refs/heads/branch-2.3 cd92913f3 -> bc4bef472 [SPARK-22850][CORE] Ensure queued events are delivered to all event queues. The code in LiveListenerBus was queueing events before start in the queues themselves; so in situations like the following: bus.post(someEvent) bus.addToEventLogQueue(listener) bus.start() "someEvent" would not be delivered to "listener" if that was the first listener in the queue, because the queue wouldn't exist when the event was posted. This change buffers the events before starting the bus in the bus itself, so that they can be delivered to all registered queues when the bus is started. Also tweaked the unit tests to cover the behavior above. Author: Marcelo VanzinCloses #20039 from vanzin/SPARK-22850. (cherry picked from commit d2cddc88eac32f26b18ec26bb59e85c6f09a8c88) Signed-off-by: Imran Rashid Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bc4bef47 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bc4bef47 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bc4bef47 Branch: refs/heads/branch-2.3 Commit: bc4bef472de0e99f74a80954d694c3d1744afe3a Parents: cd92913 Author: Marcelo Vanzin Authored: Thu Jan 4 16:19:00 2018 -0600 Committer: Imran Rashid Committed: Thu Jan 4 16:19:22 2018 -0600 -- .../spark/scheduler/LiveListenerBus.scala | 45 +--- .../spark/scheduler/SparkListenerSuite.scala| 21 + 2 files changed, 52 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bc4bef47/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 2312140..ba6387a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -62,6 +62,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) { private val queues = new CopyOnWriteArrayList[AsyncEventQueue]() + // Visible for testing. + @volatile private[scheduler] var queuedEvents = new mutable.ListBuffer[SparkListenerEvent]() + /** Add a listener to queue shared by all non-internal listeners. */ def addToSharedQueue(listener: SparkListenerInterface): Unit = { addToQueue(listener, SHARED_QUEUE) @@ -125,13 +128,39 @@ private[spark] class LiveListenerBus(conf: SparkConf) { /** Post an event to all queues. */ def post(event: SparkListenerEvent): Unit = { -if (!stopped.get()) { - metrics.numEventsPosted.inc() - val it = queues.iterator() - while (it.hasNext()) { -it.next().post(event) +if (stopped.get()) { + return +} + +metrics.numEventsPosted.inc() + +// If the event buffer is null, it means the bus has been started and we can avoid +// synchronization and post events directly to the queues. This should be the most +// common case during the life of the bus. +if (queuedEvents == null) { + postToQueues(event) + return +} + +// Otherwise, need to synchronize to check whether the bus is started, to make sure the thread +// calling start() picks up the new event. +synchronized { + if (!started.get()) { +queuedEvents += event +return } } + +// If the bus was already started when the check above was made, just post directly to the +// queues. +postToQueues(event) + } + + private def postToQueues(event: SparkListenerEvent): Unit = { +val it = queues.iterator() +while (it.hasNext()) { + it.next().post(event) +} } /** @@ -149,7 +178,11 @@ private[spark] class LiveListenerBus(conf: SparkConf) { } this.sparkContext = sc -queues.asScala.foreach(_.start(sc)) +queues.asScala.foreach { q => + q.start(sc) + queuedEvents.foreach(q.post) +} +queuedEvents = null metricsSystem.registerSource(metrics) } http://git-wip-us.apache.org/repos/asf/spark/blob/bc4bef47/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 1beb36a..da6ecb8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -48,7 +48,7 @@ class
spark git commit: [SPARK-22850][CORE] Ensure queued events are delivered to all event queues.
Repository: spark Updated Branches: refs/heads/master 93f92c0ed -> d2cddc88e [SPARK-22850][CORE] Ensure queued events are delivered to all event queues. The code in LiveListenerBus was queueing events before start in the queues themselves; so in situations like the following: bus.post(someEvent) bus.addToEventLogQueue(listener) bus.start() "someEvent" would not be delivered to "listener" if that was the first listener in the queue, because the queue wouldn't exist when the event was posted. This change buffers the events before starting the bus in the bus itself, so that they can be delivered to all registered queues when the bus is started. Also tweaked the unit tests to cover the behavior above. Author: Marcelo VanzinCloses #20039 from vanzin/SPARK-22850. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d2cddc88 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d2cddc88 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d2cddc88 Branch: refs/heads/master Commit: d2cddc88eac32f26b18ec26bb59e85c6f09a8c88 Parents: 93f92c0 Author: Marcelo Vanzin Authored: Thu Jan 4 16:19:00 2018 -0600 Committer: Imran Rashid Committed: Thu Jan 4 16:19:00 2018 -0600 -- .../spark/scheduler/LiveListenerBus.scala | 45 +--- .../spark/scheduler/SparkListenerSuite.scala| 21 + 2 files changed, 52 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d2cddc88/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 2312140..ba6387a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -62,6 +62,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) { private val queues = new CopyOnWriteArrayList[AsyncEventQueue]() + // Visible for testing. + @volatile private[scheduler] var queuedEvents = new mutable.ListBuffer[SparkListenerEvent]() + /** Add a listener to queue shared by all non-internal listeners. */ def addToSharedQueue(listener: SparkListenerInterface): Unit = { addToQueue(listener, SHARED_QUEUE) @@ -125,13 +128,39 @@ private[spark] class LiveListenerBus(conf: SparkConf) { /** Post an event to all queues. */ def post(event: SparkListenerEvent): Unit = { -if (!stopped.get()) { - metrics.numEventsPosted.inc() - val it = queues.iterator() - while (it.hasNext()) { -it.next().post(event) +if (stopped.get()) { + return +} + +metrics.numEventsPosted.inc() + +// If the event buffer is null, it means the bus has been started and we can avoid +// synchronization and post events directly to the queues. This should be the most +// common case during the life of the bus. +if (queuedEvents == null) { + postToQueues(event) + return +} + +// Otherwise, need to synchronize to check whether the bus is started, to make sure the thread +// calling start() picks up the new event. +synchronized { + if (!started.get()) { +queuedEvents += event +return } } + +// If the bus was already started when the check above was made, just post directly to the +// queues. +postToQueues(event) + } + + private def postToQueues(event: SparkListenerEvent): Unit = { +val it = queues.iterator() +while (it.hasNext()) { + it.next().post(event) +} } /** @@ -149,7 +178,11 @@ private[spark] class LiveListenerBus(conf: SparkConf) { } this.sparkContext = sc -queues.asScala.foreach(_.start(sc)) +queues.asScala.foreach { q => + q.start(sc) + queuedEvents.foreach(q.post) +} +queuedEvents = null metricsSystem.registerSource(metrics) } http://git-wip-us.apache.org/repos/asf/spark/blob/d2cddc88/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 1beb36a..da6ecb8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -48,7 +48,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
spark git commit: [SPARK-22921][PROJECT-INFRA] Bug fix in jira assigning
Repository: spark Updated Branches: refs/heads/master 66a7d6b30 -> ccda75b0d [SPARK-22921][PROJECT-INFRA] Bug fix in jira assigning Small bug fix from last pr, ran a successful merge with this code. Author: Imran RashidCloses #20117 from squito/SPARK-22921. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ccda75b0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ccda75b0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ccda75b0 Branch: refs/heads/master Commit: ccda75b0d17637abaa7ab54b68e4db8e1a16f2ce Parents: 66a7d6b Author: Imran Rashid Authored: Fri Dec 29 17:07:01 2017 -0600 Committer: Imran Rashid Committed: Fri Dec 29 17:07:01 2017 -0600 -- dev/merge_spark_pr.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ccda75b0/dev/merge_spark_pr.py -- diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index cd5dd1e..57ca840 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -242,8 +242,8 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): cur_summary = issue.fields.summary cur_assignee = issue.fields.assignee if cur_assignee is None: -cur_assignee = choose_jira_assignee(issue) -# Check again, we might not have chose an assignee +cur_assignee = choose_jira_assignee(issue, asf_jira) +# Check again, we might not have chosen an assignee if cur_assignee is None: cur_assignee = "NOT ASSIGNED!!!" else: - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org