You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by an...@apache.org on 2016/02/05 23:37:45 UTC

spark git commit: [SPARK-13002][MESOS] Send initial request of executors for dyn allocation

Repository: spark
Updated Branches:
  refs/heads/master 66e1383de -> 0bb5b7338


[SPARK-13002][MESOS] Send initial request of executors for dyn allocation

Fix for [SPARK-13002](https://issues.apache.org/jira/browse/SPARK-13002) about the initial number of executors when running with dynamic allocation on Mesos.
Instead of fixing it just for the Mesos case, made the change in `ExecutorAllocationManager`. It is already driving the number of executors running on Mesos, only no the initial value.

The `None` and `Some(0)` are internal details on the computation of resources to reserved, in the Mesos backend scheduler. `executorLimitOption` has to be initialized correctly, otherwise the Mesos backend scheduler will, either, create to many executors at launch, or not create any executors and not be able to recover from this state.

Removed the 'special case' description in the doc. It was not totally accurate, and is not needed anymore.

This doesn't fix the same problem visible with Spark standalone. There is no straightforward way to send the initial value in standalone mode.

Somebody knowing this part of the yarn support should review this change.

Author: Luc Bourlier <lu...@typesafe.com>

Closes #11047 from skyluc/issue/initial-dyn-alloc-2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0bb5b733
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0bb5b733
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0bb5b733

Branch: refs/heads/master
Commit: 0bb5b73387e60ef007b415fba69a3e1e89a4f013
Parents: 66e1383
Author: Luc Bourlier <lu...@typesafe.com>
Authored: Fri Feb 5 14:37:42 2016 -0800
Committer: Andrew Or <an...@databricks.com>
Committed: Fri Feb 5 14:37:42 2016 -0800

----------------------------------------------------------------------
 .../mesos/CoarseMesosSchedulerBackend.scala     | 13 +++++++++---
 docs/running-on-mesos.md                        | 21 +++++++++-----------
 2 files changed, 19 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/0bb5b733/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index b82196c..0a2d72f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -87,10 +87,17 @@ private[spark] class CoarseMesosSchedulerBackend(
   val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int]
 
   /**
-   * The total number of executors we aim to have. Undefined when not using dynamic allocation
-   * and before the ExecutorAllocatorManager calls [[doRequestTotalExecutors]].
+   * The total number of executors we aim to have. Undefined when not using dynamic allocation.
+   * Initially set to 0 when using dynamic allocation, the executor allocation manager will send
+   * the real initial limit later.
    */
-  private var executorLimitOption: Option[Int] = None
+  private var executorLimitOption: Option[Int] = {
+    if (Utils.isDynamicAllocationEnabled(conf)) {
+      Some(0)
+    } else {
+      None
+    }
+  }
 
   /**
    *  Return the current executor limit, which may be [[Int.MaxValue]]

http://git-wip-us.apache.org/repos/asf/spark/blob/0bb5b733/docs/running-on-mesos.md
----------------------------------------------------------------------
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 0ef1ccb..e1c87a8 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -246,18 +246,15 @@ In either case, HDFS runs separately from Hadoop MapReduce, without being schedu
 
 # Dynamic Resource Allocation with Mesos
 
-Mesos supports dynamic allocation only with coarse grain mode, which can resize the number of executors based on statistics
-of the application. While dynamic allocation supports both scaling up and scaling down the number of executors, the coarse grain scheduler only supports scaling down
-since it is already designed to run one executor per slave with the configured amount of resources. However, after scaling down the number of executors the coarse grain scheduler
-can scale back up to the same amount of executors when Spark signals more executors are needed.
-
-Users that like to utilize this feature should launch the Mesos Shuffle Service that
-provides shuffle data cleanup functionality on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's
-termination. To launch/stop the Mesos Shuffle Service please use the provided sbin/start-mesos-shuffle-service.sh and sbin/stop-mesos-shuffle-service.sh
-scripts accordingly.
-
-The Shuffle Service is expected to be running on each slave node that will run Spark executors. One way to easily achieve this with Mesos
-is to launch the Shuffle Service with Marathon with a unique host constraint.
+Mesos supports dynamic allocation only with coarse-grain mode, which can resize the number of
+executors based on statistics of the application. For general information,
+see [Dynamic Resource Allocation](job-scheduling.html#dynamic-resource-allocation).
+
+The External Shuffle Service to use is the Mesos Shuffle Service. It provides shuffle data cleanup functionality
+on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's
+termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all slave nodes, with `spark.shuffle.service.enabled` set to `true`.
+
+This can also be achieved through Marathon, using a unique host constraint, and the following command: `bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`.
 
 # Configuration
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org