You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by andrewor14 <gi...@git.apache.org> on 2014/10/10 04:41:24 UTC

[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

GitHub user andrewor14 opened a pull request:

    https://github.com/apache/spark/pull/2746

    [WIP][SPARK-3795] Heuristics for dynamically scaling executors

    This is part of a bigger effort to provide elastic scaling of executors within a Spark application ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This PR does not provide any functionality by itself; it is a skeleton that is missing a mechanism to be added later in [SPARK-3822](https://issues.apache.org/jira/browse/SPARK-3822).
    
    The design doc can be found at [SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174) (under "Heuristics for Scaling Executors". This deviates from the design in two ways: (1) the remove policy is significantly simplified by cutting out the exponential increase logic that mirrors the add policy, and (2) the configuration of the add policy is split up into `addExecutorThreshold` and `addExecutorInterval`. The main reason for this is that the user may want to configure how quickly executors are added, but want to keep the trigger condition the same.
    
    This is work-in-progress because it is missing two things: (1) logic for retrying a request to add/remove executors if the request is not fulfilled, and (2) unit tests.
    
    Comments and feedback are most welcome, but please keep in mind that this is still WIP.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/andrewor14/spark scaling-heuristics

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2746.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2746
    
----
commit b2e6dcc91897b9bd8718f1ec8d9f16b2f8075010
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-07T22:52:46Z

    Add skeleton interface for requesting / killing executors

commit 6f1fa66f0ce0b141a934ade3235d873e6e1d8a60
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-08T23:06:23Z

    First cut implementation of adding executors dynamically
    
    This provides a framework that keeps track of when to add and
    remove executors, and how many. The add part of this framework
    is fully implemented minus the backend, i.e. we do not yet
    actually ask the cluster manager for more executors, as that
    depends on the backend implementation.

commit 4077ae21a05dc1af9985bf2cf419d5ecff6fde99
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-08T23:41:01Z

    Minor code re-organization

commit 20ec6b9955113d7dedc915825b54bbf36dd820e0
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-09T02:54:48Z

    First cut implementation of removing executors dynamically
    
    This sets a timer on each executor every time it is launched or
    has finished a task. The remove executor timer is then started
    as soon as one of the idle executors is triggered. An idle
    executor timer is cancelled when the executor starts running
    a task again, and the remove executor timer is cancelled when
    there are no more idle executors.
    
    An important TODO is to deal with synchronization. I have
    witnessed multiple remove executor timers being started due
    to the lack of synchronization.

commit ae5b64a1d2572321931e95cb0297a7e3636470c4
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-09T20:10:58Z

    Add synchronization
    
    Synchronization is applied somewhat profusely in both classes.
    This is because the task scheduler is single-threaded, and normally
    does not schedule multiple tasks concurrently. Synchronization is
    still needed, however, when executors are added or removed, but
    these are relatively rare events so it is OK to synchronize this way.

commit 1cc84446d18d1b1078e98260856ffc278239d614
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-09T20:22:20Z

    Minor wording change

commit 89019008088dd8cc34baf1c9e65ceaba66889f62
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-09T23:05:46Z

    Simplify remove policy + change the semantics of add policy
    
    With the latest changes, exponential increase only applies to the
    add policy. Then, the add threshold and the add interval are
    decoupled into separate configurations. The remove policy is made
    simpler: an executor is removed as soon as it is marked as idle.
    
    This also updates the comments to explain why we use exponential
    increase in the number of executors when adding executors.

commit 6c48ab003af8b809ccc05b58e12148c2e89fcfb4
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-09T23:13:40Z

    Update synchronization comment

commit 67c03c7ca4781d62da41b995e59014c374931e99
Author: Andrew Or <an...@gmail.com>
Date:   2014-10-10T02:26:52Z

    Correct semantics of adding executors + update comments
    
    Previously, we just kept asking for more executors whether or not
    we are granted the ones we have asked for in the past. This is
    extremely not cautious and could lead to an overflow of add executor
    requests, if the cluster manager cannot accommodate all the requests,
    for instance. This commit adds the correct guards against attempting
    to request more executors before the pending ones have all registered.
    
    An important TODO at this point is to retry these requests if
    they're not immediately granted. We can do this after a timeout.
    The tricky thing there is that if the retry is too eager then we
    may end up exceeding the upper bound on the number of executors.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60552189
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22272/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59143487
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21753/consoleFull) for   PR 2746 at commit [`baaa403`](https://github.com/apache/spark/commit/baaa403b16d10107abeeb1fba721b2c301d2c0aa).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19230787
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  def initialize(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +
    +              // Expire any outstanding pending add requests that have timed out
    +              if (pendingAddExpirationTime != NOT_STARTED && now >= pendingAddExpirationTime) {
    +                logDebug(s"Expiring all pending add requests because they have " +
    +                  s"not been fulfilled after $pendingAddTimeoutSeconds seconds")
    +                numExecutorsPendingToAdd = 0
    +                pendingAddExpirationTime = NOT_STARTED
    +              }
    +
    +              // Expire any outstanding pending remove requests that have timed out
    +              pendingRemoveExpirationTimes.foreach { case (executorId, expirationTime) =>
    +                if (now > expirationTime) {
    +                  logDebug(s"Expiring pending request to remove executor $executorId because " +
    +                    s"it has not been fulfilled after $pendingRemoveTimeoutSeconds seconds")
    +                  executorsPendingToRemove.remove(executorId)
    +                  pendingRemoveExpirationTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    --- End diff --
    
    I think it's even clearer if we avoid using `math.min`:
    ```
    val actualNumExecutorsToAdd =
      if (numExistingExecutors + numExecutorsToAdd > maxNumExecutors) {
        maxNumExecutors - numExistingExecutors
      } else {
        numExecutorsToAdd
      }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60972887
  
    LGTM, although I'll probably go through it again later (but don't wait for that). I think we should file a sub-task to look at static vs. dynamic allocation user options and make sure we do something sane.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19440194
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -329,6 +329,15 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
         } else None
       }
     
    +  // Optionally scale number of executors dynamically based on workload
    +  private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] =
    --- End diff --
    
    Can you add a comment that we keep a link to this only for testing purposes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60862789
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22412/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61006673
  
      [Test build #22478 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22478/consoleFull) for   PR 2746 at commit [`c0becc4`](https://github.com/apache/spark/commit/c0becc47611d0e0bbe4c630ab8ab4c52463db616).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19430468
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    --- End diff --
    
    Ah, I see, so this matters for yarn-client mode.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60477158
  
      [Test build #22207 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22207/consoleFull) for   PR 2746 at commit [`19c6c4b`](https://github.com/apache/spark/commit/19c6c4b7ad6b64f65475dc494a91c75226b72410).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging `
      * `  class JobUIData(`
      * `public final class JavaStatusAPIDemo `
      * `  public static final class IdentityWithDelay<T> implements Function<T, T> `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60470225
  
    Hi all, I have implemented the relevant changes in #2840. The interfaces there are very similar to what we have discussed. A major difference is that the scheduler backend specifies the total number of executors desired for the application rather than the pending number of executors. This takes care of the race condition that @sryza pointed out above. In practice, I have observed this race condition somewhat frequently before these changes went in.
    
    (Note that we can't rely on `YarnAllocator` to handle the over-allocation scenarios as @vanzin suggested. This is because by the time the AM gets a pending request from the driver, it has no way of knowing whether the request is sent before or after any prior requests have been fulfilled.)
    
    I intend to merge #2840 before this patch, so I would really appreciate it if all of you could review that one first. I have tested that one quite rigorously and I no longer observe any of the race conditions we used to see with the old semantics.
    
    @sryza @vanzin @pwendell @kayousterhout


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19389346
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    --- End diff --
    
    These should be documented in the configuration page, unless that's planned for a later patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60799644
  
    One question: is `spark.dynamicAllocation.minExecutors` necessary? Can't that be set to the initial number of executors requested by the user (i.e. `spark-submit --num-executors`)? That seems more logical to me, otherwise you have this weird situation:
    
    * --num-executors = 1
    * spark.dynamicAllocation.minExecutors = 10
    * spark.dynamicAllocation.maxExecutors = 20
    
    After the dynamic allocation code runs and your app goes back to idle, you'll end up with 11 (or 10? didn't follow the math closely) executors running.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60547191
  
      [Test build #22271 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22271/consoleFull) for   PR 2746 at commit [`254c958`](https://github.com/apache/spark/commit/254c9584d67f674efe828e07ff6c4222e0386c4d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19194607
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    --- End diff --
    
    I also wonder if the word "timer" should not be used here. To me "timer" means a clock that is counting up or down as time passes. Here it means an absolute time at which an event should trigger. I think in reality "timer" can mean both things (http://www.merriam-webster.com/dictionary/timer), but it might be good to avoid the name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60339499
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22105/consoleFull) for   PR 2746 at commit [`092d1fd`](https://github.com/apache/spark/commit/092d1fdc9f8f450a447a3c8292a78eef9937ab3a).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18743600
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    --- End diff --
    
    All of those sound good to me.  The second one if I had to choose.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61025251
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22485/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60197171
  
    So yeah it internally decrements the pending number to 8.  The app can and is expected to infer YARN has decremented the counter.  Maybe TMI, but for getting a grasp on it, it might be helpful to understand the race conditions this approach exposes - i.e. there are situations where YARN can overallocate.  For example imagine you requested 10 and then you decide you want 11. YARN just got 2 for you and decremented its counter to 8.  You might tell YARN you want 11 before finding out about the 2 YARN is giving to you, which means you would overwrite the 8 with 11.  In the brief period before you can go back to YARN and tell it you only want 9 now, it could conceivably give you 11 containers, for a total of 13, which is more than you ever asked for.  The app is expected to handle these situations and release allocated containers that it doesn't need.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19452708
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if the time then has exceeded any of the
    +   * add and remove times that are set. If so, it triggers the corresponding action.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add time has expired, add executors and refresh the add time
    +              if (addTime != NOT_SET && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Starting timer to add more executors (to " +
    +                  s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +                addTime += sustainedSchedulerBacklogTimeout * 1000
    +              }
    +
    +              // If any remove time has expired, remove the corresponding executor
    +              removeTimes.foreach { case (executorId, expireTime) =>
    +                if (now > expireTime) {
    +                  removeExecutor(executorId)
    +                  removeTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    --- End diff --
    
    Rather than expose all these methods for testing, can you use `PrivateMethodTester`?
    
    http://doc.scalatest.org/1.7/org/scalatest/PrivateMethodTester.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58736603
  
    @sryza Thanks for the comments. Unfortunately I have made significant changes recently and much of the code is now outdated. In my original design I went with a callback-based approach rather than a polling approach because I wanted the semantics of the former. In particular, I wanted to add/remove executors only if the respective condition has been satisfied without interruption for a certain duration, and this is difficult to guarantee precisely with polling.
    
    HOWEVER, the significant advantage in polling is that we only need one extra thread rather than one for each timer. I am convinced that the latter approach is probably both simpler and more scalable, and I'll likely make the changes shortly.
    
    Please hold off reviewing this PR for now until I make the relevant changes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59281945
  
    > This still uses the exponential increase policy where each round stops and waits until the executors are actually granted by the cluster manager before continuing.
    
    To be clear, I don't mean to argue that we need to incorporate more advanced schemes into this PR.  Just that we should name the configs in such a way that adding them later won't look weird.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377793
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd *= 2
    +      numExecutorsPending += actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are " +
    +        s"only $minNumExecutors executor(s) left, which is the limit")
    +      return
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be triggered
    +   * in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in every round
    +   * thereafter. This is called when the scheduler receives new pending tasks.
    +   */
    +  def startAddTimer(): Unit = synchronized {
    --- End diff --
    
    Could these functions be renamed to not convey information about internal details? The main information that the listener needs to convey to this class are things like when executors start tasks, when the queue is idle, etc. These function names all are specific to internal timers, etc.
    
    What bout having the external interface look like this:
    ```
    def schedulerQueueBacklogged()
    def schedulerQueueEmpty()
    def executorIdle(executorId: String)
    def executorBusy(executorId: String)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61017681
  
      [Test build #22485 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22485/consoleFull) for   PR 2746 at commit [`340f3b1`](https://github.com/apache/spark/commit/340f3b19184b9709020306a84872068a5d5ea507).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59870868
  
    With regard to the configuration barrier, I actually think the exposed configs are pretty straightforward. I think even the inexperienced user can reason about the number of executors being scaled up and down within a custom range. All the user needs to set is the min and the max, and everything else is optional.
    
    That said, I should clarify that I am not discounting this other policy once and for all. I do believe in its merits, but I think the default scaling policies in Spark should be as simple as possible, both in terms of implementation and semantics. I am open to introducing it as a pluggable policy in a future release, but I prefer to use a different approach as the first-cut implementation for the aforementioned reasons.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19194672
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    --- End diff --
    
    will this be replaced by directly adding against the SparkContext? I think that would be nicer since that way it's clear how users can use this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60709625
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22328/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60353109
  
    Hey Andrew - these heuristics look good to me at this point. I think the remaining things are doing this via a listener and using the SparkContext API's. Also, we should of course add tests for this and try to test the corner cases, etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59556002
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21859/consoleFull) for   PR 2746 at commit [`9d516cc`](https://github.com/apache/spark/commit/9d516ccdcef852644a5a3423ad7f6dd2f5820dad).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19452557
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if the time then has exceeded any of the
    +   * add and remove times that are set. If so, it triggers the corresponding action.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add time has expired, add executors and refresh the add time
    +              if (addTime != NOT_SET && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Starting timer to add more executors (to " +
    +                  s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +                addTime += sustainedSchedulerBacklogTimeout * 1000
    +              }
    +
    +              // If any remove time has expired, remove the corresponding executor
    +              removeTimes.foreach { case (executorId, expireTime) =>
    +                if (now > expireTime) {
    +                  removeExecutor(executorId)
    +                  removeTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already ${executorIds.size} " +
    +        s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
    +        s"tasks are backlogged (new desired total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to remove the given executor.
    +   * Return whether the request is received. Exposed for testing.
    +   */
    +  def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId!")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Attempted to remove executor $executorId " +
    +        s"when it is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left (limit $minNumExecutors)")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added. Exposed for testing.
    +   */
    +  def onExecutorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(onExecutorIdle)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented number of pending executors ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed. Exposed for testing.
    +   */
    +  def onExecutorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Executor $executorId is no longer pending to " +
    +          s"be removed (${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler receives new pending tasks.
    +   * This sets a time in the future that decides when executors should be added
    +   * if it is not already set. Exposed for testing.
    +   */
    +  def onSchedulerBacklogged(): Unit = synchronized {
    +    if (addTime == NOT_SET) {
    +      logDebug(s"Starting timer to add executors because pending tasks " +
    +        s"are building up (to expire in $schedulerBacklogTimeout seconds)")
    +      addTime = System.currentTimeMillis + schedulerBacklogTimeout * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler queue is drained.
    +   * This resets all variables used for adding executors. Exposed for testing.
    +   */
    +  def onSchedulerQueueEmpty(): Unit = synchronized {
    +    logDebug(s"Clearing timer to add executors because there are no more pending tasks")
    +    addTime = NOT_SET
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is no longer running any tasks.
    +   * This sets a time in the future that decides when this executor should be removed if
    +   * the executor is not already marked as idle. Exposed for testing.
    +   */
    +  def onExecutorIdle(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
    +        s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is now running a task.
    +   * This resets all variables used for removing this executor. Exposed for testing.
    +   */
    +  def onExecutorBusy(executorId: String): Unit = synchronized {
    +    logDebug(s"Clearing idle timer for $executorId because it is now running a task")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /* --------------------------- *
    +   | Getters exposed for testing |
    +   * --------------------------- */
    +
    +  def getNumExecutorsToAdd: Int = numExecutorsToAdd
    +  def getNumExecutorsPending: Int = numExecutorsPending
    +  def getExecutorsPendingToRemove: collection.Set[String] = executorsPendingToRemove
    +  def getExecutorIds: collection.Set[String] = executorIds
    +  def getAddTime: Long = addTime
    +  def getRemoveTimes: collection.Map[String, Long] = removeTimes
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  val NOT_SET = Long.MaxValue
    +}
    +
    +/**
    + * A listener that notifies the given allocation manager of when to add and remove executors.
    + */
    +private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    --- End diff --
    
    I would add something here that says:
    
    ```
    This class is intentionally conservative in its assumptions about the relative ordering and
    consistency of events returned by the listener. It does not account for speculated tasks to
    simplify the logic.
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2746


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19516668
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,421 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  verifyBounds()
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Verify that the lower and upper bounds on the number of executors are valid.
    +   * If not, throw an appropriate exception.
    +   */
    +  private def verifyBounds(): Unit = {
    +    if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +    }
    +    if (minNumExecutors > maxNumExecutors) {
    +      throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
    +        s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
    +    }
    +  }
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +    // TODO: start at `maxNumExecutors` once SPARK-3822 goes in
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          try {
    +            maybeAddAndRemove()
    +          } catch {
    +            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * If the add time has expired, request new executors and refresh the add time.
    +   * If the remove time for an existing executor has expired, kill the executor.
    +   * This is factored out into its own method for testing.
    +   */
    +  private def maybeAddAndRemove(now: Long = System.currentTimeMillis): Unit = synchronized {
    --- End diff --
    
    Could you call this `process` or something more general? This name is a bit akward.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19195056
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  def initialize(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +
    +              // Expire any outstanding pending add requests that have timed out
    +              if (pendingAddExpirationTime != NOT_STARTED && now >= pendingAddExpirationTime) {
    +                logDebug(s"Expiring all pending add requests because they have " +
    +                  s"not been fulfilled after $pendingAddTimeoutSeconds seconds")
    +                numExecutorsPendingToAdd = 0
    +                pendingAddExpirationTime = NOT_STARTED
    +              }
    +
    +              // Expire any outstanding pending remove requests that have timed out
    +              pendingRemoveExpirationTimes.foreach { case (executorId, expirationTime) =>
    +                if (now > expirationTime) {
    +                  logDebug(s"Expiring pending request to remove executor $executorId because " +
    +                    s"it has not been fulfilled after $pendingRemoveTimeoutSeconds seconds")
    +                  executorsPendingToRemove.remove(executorId)
    +                  pendingRemoveExpirationTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    --- End diff --
    
    Here it would be good to log both the number of pending and the current number.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18733056
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors
    +  private val addExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60) // s
    +  private val addExecutorInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addExecutorThreshold) // s
    +  private val removeExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 300) // s
    +
    +  // Timers that keep track of when to add and remove executors
    +  private var addExecutorTimer: Option[Timer] = None
    +  private val removeExecutorTimers: mutable.Map[String, Timer] = new mutable.HashMap[String, Timer]
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // The number of pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private var numExecutorsPendingToRemove = 0
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String] ++= scheduler.executorIdToHost.keys
    +
    +  // Start idle timer for all new executors
    +  synchronized { executorIds.foreach(startRemoveExecutorTimer) }
    +
    +  /**
    +   * Start the add executor timer if it does not already exist.
    +   * This is called when a new pending task is added. The add is then triggered
    +   * if the pending tasks queue is not drained in `addExecutorThreshold` seconds.
    +   */
    +  def startAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorThreshold)
    +
    +  /**
    +   * Restart the add executor timer.
    +   * This is called when the previous add executor timer has expired but not canceled. The add
    +   * is then triggered again if all pending executors from the previous round have registered,
    +   * and the pending tasks queue is still not drained in `addExecutorInterval` seconds.
    +   */
    +  private def restartAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorInterval)
    +
    +  /**
    +   * Start the add executor timer using the given delay if the timer does not already exist.
    +   */
    +  private def startAddExecutorTimer(timerDelaySeconds: Long): Unit = {
    +    addExecutorTimer.synchronized {
    +      if (addExecutorTimer.isEmpty) {
    +        logDebug(s"Starting add executor timer (to expire in $timerDelaySeconds seconds)")
    +        addExecutorTimer = Some(new Timer)
    +        addExecutorTimer.get.schedule(
    +          new AddExecutorTimerTask(numExecutorsToAdd), timerDelaySeconds * 1000)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor if the timer does not already exist.
    +   * This is called when the executor initially registers with the driver or finishes running
    +   * a task. The removal is then triggered if the executor stays idle (i.e. not running a task)
    +   * for `removeExecutorThreshold` seconds.
    +   */
    +  def startRemoveExecutorTimer(executorId: String): Unit = {
    +    removeExecutorTimers.synchronized {
    +      if (!removeExecutorTimers.contains(executorId)) {
    +        logDebug(s"Starting idle timer for executor $executorId " +
    +          s"(to expire in $removeExecutorThreshold seconds)")
    +        removeExecutorTimers(executorId) = new Timer
    +        removeExecutorTimers(executorId).schedule(
    +          new RemoveExecutorTimerTask(executorId), removeExecutorThreshold * 1000)
    +      }
    +    }
    +    // Acquire a more general lock here because we might mutate `executorId`
    +    synchronized {
    +      if (!executorIds.contains(executorId)) {
    +        logWarning(s"Started idle timer for unknown executor $executorId.")
    +        executorIds.add(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that adds executors.
    +   * This is called when the pending task queue is drained.
    +   */
    +  def cancelAddExecutorTimer(): Unit = addExecutorTimer.synchronized {
    +    addExecutorTimer.foreach { timer =>
    +      logDebug("Canceling add executor timer because task queue is drained!")
    +      timer.cancel()
    +      numExecutorsToAdd = 1
    +      addExecutorTimer = None
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that removes the given executor.
    +   * This is called when the executor is no longer idle.
    +   */
    +  def cancelRemoveExecutorTimer(executorId: String): Unit = removeExecutorTimers.synchronized {
    +    if (removeExecutorTimers.contains(executorId)) {
    +      logDebug(s"Canceling idle timer for executor $executorId.")
    +      removeExecutorTimers(executorId).cancel()
    +      removeExecutorTimers.remove(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Negotiate with the scheduler backend to add new executors.
    +   * This ensures the resulting number of executors is correctly constrained by the upper bound.
    +   * Return the number of executors actually requested.
    +   */
    +  private def addExecutors(numExecutorsRequested: Int): Int = synchronized {
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    val numExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsRequested <= maxNumExecutors) {
    +        numExecutorsRequested
    +      } else {
    +        // Add just enough to reach `maxNumExecutors`
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newNumExecutors = numExistingExecutors + numExecutorsToAdd
    +
    +    if (numExecutorsToAdd > 0) {
    +      getCoarseGrainedBackend.foreach { backend =>
    +        logInfo(s"Pending tasks are building up! " +
    +          s"Adding $numExecutorsToAdd new executor(s) (new total is $newNumExecutors).")
    +        numExecutorsPendingToAdd += numExecutorsToAdd
    +        backend.requestExecutors(numExecutorsToAdd)
    +        return numExecutorsToAdd
    +      }
    +    } else {
    +      logDebug(s"Not adding executors because there are already $maxNumExecutors executors, " +
    +        s"which is the limit.")
    +    }
    +    0
    +  }
    +
    +  /**
    +   * Negotiate with the scheduler backend to remove existing executors.
    +   * This ensures the resulting number of executors is correctly constrained by the lower bound.
    +   * Return whether the request to remove the executor is actually sent.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    val numExistingExecutors = executorIds.size - numExecutorsPendingToRemove
    +    if (numExistingExecutors - 1 >= minNumExecutors) {
    +      getCoarseGrainedBackend.foreach { backend =>
    +        logInfo(s"Removing executor $executorId because it has been idle for " +
    +          s"$removeExecutorThreshold seconds (new total is ${numExistingExecutors - 1}).")
    +        numExecutorsPendingToRemove += 1
    +        backend.killExecutor(executorId)
    +        return true
    +      }
    +    } else {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit.")
    +    }
    +    false
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered.")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decrementing pending executors to add (now at $numExecutorsPendingToAdd).")
    +      }
    +      executorIds.add(executorId)
    +      startRemoveExecutorTimer(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      logDebug(s"Existing executor $executorId has been removed.")
    +      executorIds.remove(executorId)
    +      if (numExecutorsPendingToRemove > 0) {
    +        numExecutorsPendingToRemove -= 1
    +        logDebug(s"Decrementing pending executors to remove (now at $numExecutorsPendingToRemove).")
    +      }
    +    } else {
    +      logWarning(s"Not removing unknown executor $executorId")
    +    }
    +  }
    +
    +  /**
    +   * Return the backend as a CoarseGrainedSchedulerBackend if possible.
    +   * Otherwise, guard against the use of this feature either before the backend has initialized,
    +   * or because the scheduler is running in fine-grained mode. In the latter case, the executors
    +   * are already dynamically allocated by definition, so an appropriate exception is thrown.
    +   */
    +  private def getCoarseGrainedBackend: Option[CoarseGrainedSchedulerBackend] = {
    +    scheduler.backend match {
    +      case b: CoarseGrainedSchedulerBackend => Some(b)
    +      case null =>
    +        logWarning("Scheduler backend not initialized yet for dynamically scaling executors!")
    +        None
    +      case _ =>
    +        throw new SparkException("Dynamic allocation of executors is not applicable to " +
    +          "fine-grained schedulers. Please set spark.dynamicAllocation.enabled to false.")
    +    }
    +  }
    +
    +  /**
    +   * A timer task that adds the given number of executors.
    +   *
    +   * This task does not request new executors until the ones pending from the previous round have
    +   * all registered. Then, if the number of executors requested is as expected (i.e. the upper
    +   * bound is not reached), the number to request next round increases exponentially. Finally,
    +   * after requesting executors, this restarts the add executor timer unless the timer is canceled.
    +   */
    +  private class AddExecutorTimerTask(_numExecutorsToAdd: Int) extends TimerTask {
    +    override def run(): Unit = {
    +      // Whether we have successfully requested the expected number of executors
    +      var success = false
    +
    +      synchronized {
    +        // Do not add executors until those requested in the previous round have registered
    +        if (numExecutorsPendingToAdd == 0) {
    +          val numExecutorsAdded = addExecutors(_numExecutorsToAdd)
    +          success = numExecutorsAdded == _numExecutorsToAdd
    +        } else {
    +          logInfo(s"Not adding new executors until all $numExecutorsPendingToAdd pending " +
    +            "executor(s) have registered.")
    +        }
    +      }
    +
    +      addExecutorTimer.synchronized {
    +        // Do this check in case the timer has been canceled in the mean time
    +        if (addExecutorTimer.isDefined) {
    +          numExecutorsToAdd = if (success) { _numExecutorsToAdd * 2 } else 1
    --- End diff --
    
    Nit: take out these curly braces?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19515984
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -0,0 +1,407 @@
    +/*
    + * 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
    +
    +import org.scalatest.FunSuite
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.scheduler._
    +import org.apache.spark.storage.BlockManagerId
    +
    +class ExecutorAllocationManagerSuite extends FunSuite {
    --- End diff --
    
    Thanks, more to come :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60198472
  
    Yep - that's exactly what I was wondering about. If YARN doesn't expose the
    internal counter it seems like there is a race (actually even if it does
    expose it, there still is a minor race where you could read it and then
    reset it but it changes in the middle). I guess we just live with it...
    
    On Wed, Oct 22, 2014 at 11:21 PM, Sandy Ryza <no...@github.com>
    wrote:
    
    > So yeah it internally decrements the pending number to 8. The app can and
    > is expected to infer YARN has decremented the counter. Maybe TMI, but for
    > getting a grasp on it, it might be helpful to understand the race
    > conditions this approach exposes - i.e. there are situations where YARN can
    > overallocate. For example imagine you requested 10 and then you decide you
    > want 11. YARN just got 2 for you and decremented its counter to 8. You
    > might tell YARN you want 11 before finding out about the 2 YARN is giving
    > to you, which means you would overwrite the 8 with 11. In the brief period
    > before you can go back to YARN and tell it you only want 9 now, it could
    > conceivably give you 11 containers, for a total of 13, which is more than
    > you ever asked for. The app is expected to handle these situations and
    > release allocated containers that it doesn't need.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/2746#issuecomment-60197171>.
    >


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18923301
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -110,19 +110,26 @@ private[spark] class TaskSetManager(
       // the one that it was launched from, but gets removed from them later.
       private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
    +  // List 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.
    +  // List of pending tasks for each rack -- similar to the above.
       private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set containing pending tasks with no locality preferences.
    +  // List of pending tasks with no locality preferences.
       var pendingTasksWithNoPrefs = new ArrayBuffer[Int]
     
    -  // Set containing all pending tasks (also used as a stack, as above).
    +  // List of all pending tasks (also used as a stack, as above).
       val allPendingTasks = new ArrayBuffer[Int]
     
    +  // Set of pending tasks used to keep track of whether more executors are needed
    --- End diff --
    
    Good question. I spent a long time perusing whether we could just use `allPendingTasks`. Apparently we remove pending tasks there lazily (when we call `findTaskFromList`) instead of as soon as it's been scheduled. In most cases, I've noticed that this is actually usually emptied all at once because we usually grab the task from other lists with more specific locality preferences. This means whatever's in there is actually not representative of what's actually still pending to be scheduled, so we can't use it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18923728
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    --- End diff --
    
    is this data structure thread safe (i.e. safe to be accessed outside of taskschedulerimpl)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19557841
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,453 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  verifyBounds()
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  // Clock used to schedule when executors should be added and removed
    +  private var clock: Clock = new RealClock
    +
    +  /**
    +   * Verify that the lower and upper bounds on the number of executors are valid.
    +   * If not, throw an appropriate exception.
    +   */
    +  private def verifyBounds(): Unit = {
    +    if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +    }
    +    if (minNumExecutors == 0 || maxNumExecutors == 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!")
    +    }
    +    if (minNumExecutors > maxNumExecutors) {
    +      throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
    +        s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
    +    }
    +  }
    +
    +  /**
    +   * Use a different clock for this allocation manager. This is mainly used for testing.
    +   */
    +  def setClock(newClock: Clock): Unit = {
    +    clock = newClock
    +  }
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +    // TODO: start at `maxNumExecutors` once SPARK-3822 goes in
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          try {
    +            schedule()
    +          } catch {
    +            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * If the add time has expired, request new executors and refresh the add time.
    +   * If the remove time for an existing executor has expired, kill the executor.
    +   * This is factored out into its own method for testing.
    +   */
    +  private def schedule(): Unit = synchronized {
    +    val now = clock.getTime
    +    if (addTime != NOT_SET && now >= addTime) {
    +      addExecutors()
    +      logDebug(s"Starting timer to add more executors (to " +
    +        s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +      addTime += sustainedSchedulerBacklogTimeout * 1000
    +    }
    +
    +    removeTimes.foreach { case (executorId, expireTime) =>
    +      if (now >= expireTime) {
    +        removeExecutor(executorId)
    +        removeTimes.remove(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested.
    +   */
    +  private def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already ${executorIds.size} " +
    +        s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
    +        s"tasks are backlogged (new desired total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to remove the given executor.
    +   * Return whether the request is received.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId!")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Attempted to remove executor $executorId " +
    +        s"when it is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left (limit $minNumExecutors)")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  private def onExecutorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(onExecutorIdle)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented number of pending executors ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  private def onExecutorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Executor $executorId is no longer pending to " +
    +          s"be removed (${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler receives new pending tasks.
    +   * This sets a time in the future that decides when executors should be added
    +   * if it is not already set.
    +   */
    +  private def onSchedulerBacklogged(): Unit = synchronized {
    +    if (addTime == NOT_SET) {
    +      logDebug(s"Starting timer to add executors because pending tasks " +
    +        s"are building up (to expire in $schedulerBacklogTimeout seconds)")
    +      addTime = clock.getTime + schedulerBacklogTimeout * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler queue is drained.
    +   * This resets all variables used for adding executors.
    +   */
    +  private def onSchedulerQueueEmpty(): Unit = synchronized {
    +    logDebug(s"Clearing timer to add executors because there are no more pending tasks")
    +    addTime = NOT_SET
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is no longer running any tasks.
    +   * This sets a time in the future that decides when this executor should be removed if
    +   * the executor is not already marked as idle.
    +   */
    +  private def onExecutorIdle(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
    +        s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = clock.getTime + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is now running a task.
    +   * This resets all variables used for removing this executor.
    +   */
    +  private def onExecutorBusy(executorId: String): Unit = synchronized {
    +    logDebug(s"Clearing idle timer for $executorId because it is now running a task")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /**
    +   * A listener that notifies the given allocation manager of when to add and remove executors.
    +   *
    +   * This class is intentionally conservative in its assumptions about the relative ordering
    +   * and consistency of events returned by the listener. For simplicity, it does not account
    +   * for speculated tasks.
    +   */
    +  private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +    extends SparkListener {
    +
    +    private val stageIdToNumTasks = new mutable.HashMap[Int, Int]
    +    private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +    private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +    override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
    +      synchronized {
    +        val stageId = stageSubmitted.stageInfo.stageId
    +        val numTasks = stageSubmitted.stageInfo.numTasks
    +        stageIdToNumTasks(stageId) = numTasks
    +        allocationManager.onSchedulerBacklogged()
    +      }
    +    }
    +
    +    override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
    +      synchronized {
    +        val stageId = stageCompleted.stageInfo.stageId
    +        stageIdToNumTasks -= stageId
    +        stageIdToTaskIndices -= stageId
    +
    +        // If this is the last stage with pending tasks, mark the scheduler queue as empty
    +        // This is needed in case the stage is aborted for any reason
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +    }
    +
    +    override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {
    +      val stageId = taskStart.stageId
    +      val taskId = taskStart.taskInfo.taskId
    +      val taskIndex = taskStart.taskInfo.index
    +      val executorId = taskStart.taskInfo.executorId
    +
    +      // If this is the last pending task, mark the scheduler queue as empty
    +      stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex
    +      val numTasksScheduled = stageIdToTaskIndices(stageId).size
    +      val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1)
    +      if (numTasksScheduled == numTasksTotal) {
    +        // No more pending tasks for this stage
    +        stageIdToNumTasks -= stageId
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +
    +      // Mark the executor on which this task is scheduled as busy
    +      executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId
    +      allocationManager.onExecutorBusy(executorId)
    +    }
    +
    +    override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
    +      val executorId = taskEnd.taskInfo.executorId
    +      val taskId = taskEnd.taskInfo.taskId
    +
    +      // If the executor is no longer running scheduled any tasks, mark it as idle
    +      if (executorIdToTaskIds.contains(executorId)) {
    +        executorIdToTaskIds(executorId) -= taskId
    +        if (executorIdToTaskIds(executorId).isEmpty) {
    +          executorIdToTaskIds -= executorId
    +          allocationManager.onExecutorIdle(executorId)
    +        }
    +      }
    +    }
    +
    +    override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = {
    +      val executorId = blockManagerAdded.blockManagerId.executorId
    +      if (executorId != "<driver>") {
    +        allocationManager.onExecutorAdded(executorId)
    +      }
    +    }
    +
    +    override def onBlockManagerRemoved(
    +        blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = {
    +      allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId)
    +    }
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  val NOT_SET = Long.MaxValue
    +}
    +
    +private trait Clock {
    +  def getTime: Long
    +}
    +
    +private class RealClock extends Clock {
    +  override def getTime: Long = System.currentTimeMillis
    --- End diff --
    
    For paranoia, I'd use `System.nanoTime()` instead. `System.currentTimeMillis()` can go backwards...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59136585
  
    Hey Jenkins, test this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922277
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    --- End diff --
    
    Yeah this is a broader issue I have filed at https://issues.apache.org/jira/browse/SPARK-3859. I thought about adding `Seconds` to the end but it makes the name really long. I'm still on the fence about that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18912692
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    --- End diff --
    
    just so I understand - are the semantics of this that once we hit a state of continued backlog (e.g. we are adding executors, we will continue to add executors every M seconds)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19561633
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -0,0 +1,661 @@
    +/*
    + * 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
    +
    +import org.scalatest.{FunSuite, PrivateMethodTester}
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.scheduler._
    +import org.apache.spark.storage.BlockManagerId
    +
    +/**
    + * Test add and remove behavior of ExecutorAllocationManager.
    + */
    +class ExecutorAllocationManagerSuite extends FunSuite {
    +  import ExecutorAllocationManager._
    +  import ExecutorAllocationManagerSuite._
    +
    +  test("verify min/max executors") {
    +    // No min or max
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +    intercept[SparkException] { new SparkContext(conf) }
    +
    +    // Only min
    +    val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
    +    intercept[SparkException] { new SparkContext(conf1) }
    +
    +    // Only max
    +    val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
    +    intercept[SparkException] { new SparkContext(conf2) }
    +
    +    // Both min and max, but min > max
    +    intercept[SparkException] { createSparkContext(2, 1) }
    +
    +    // Both min and max, and min == max
    +    val sc1 = createSparkContext(1, 1)
    +    assert(sc1.executorAllocationManager.isDefined)
    +    sc1.stop()
    +
    +    // Both min and max, and min < max
    +    val sc2 = createSparkContext(1, 2)
    +    assert(sc2.executorAllocationManager.isDefined)
    +    sc2.stop()
    +  }
    +
    +  test("starting state") {
    +    val sc = createSparkContext()
    +    val manager = sc.executorAllocationManager.get
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(executorIds(manager).isEmpty)
    +    assert(addTime(manager) === ExecutorAllocationManager.NOT_SET)
    +    assert(removeTimes(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test("add executors") {
    +    val sc = createSparkContext(1, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Keep adding until the limit is reached
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 1)
    +    assert(numExecutorsPending(manager) === 1)
    +    assert(numExecutorsToAdd(manager) === 2)
    +    assert(addExecutors(manager) === 2)
    +    assert(numExecutorsPending(manager) === 3)
    +    assert(numExecutorsToAdd(manager) === 4)
    +    assert(addExecutors(manager) === 4)
    +    assert(numExecutorsPending(manager) === 7)
    +    assert(numExecutorsToAdd(manager) === 8)
    +    assert(addExecutors(manager) === 3) // reached the limit of 10
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +
    +    // Register previously requested executors
    +    onExecutorAdded(manager, "first")
    +    assert(numExecutorsPending(manager) === 9)
    +    onExecutorAdded(manager, "second")
    +    onExecutorAdded(manager, "third")
    +    onExecutorAdded(manager, "fourth")
    +    assert(numExecutorsPending(manager) === 6)
    +    onExecutorAdded(manager, "first") // duplicates should not count
    +    onExecutorAdded(manager, "second")
    +    assert(numExecutorsPending(manager) === 6)
    +
    +    // Try adding again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    sc.stop()
    +  }
    +
    +  test("remove executors") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +    (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) }
    +
    +    // Keep removing until the limit is reached
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(removeExecutor(manager, "1"))
    +    assert(executorsPendingToRemove(manager).size === 1)
    +    assert(executorsPendingToRemove(manager).contains("1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(removeExecutor(manager, "3"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).contains("2"))
    +    assert(executorsPendingToRemove(manager).contains("3"))
    +    assert(!removeExecutor(manager, "100")) // remove non-existent executors
    +    assert(!removeExecutor(manager, "101"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(!removeExecutor(manager, "6")) // reached the limit of 5
    +    assert(executorsPendingToRemove(manager).size === 5)
    +    assert(executorsPendingToRemove(manager).contains("4"))
    +    assert(executorsPendingToRemove(manager).contains("5"))
    +    assert(!executorsPendingToRemove(manager).contains("6"))
    +
    +    // Kill executors previously requested to remove
    +    onExecutorRemoved(manager, "1")
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("1"))
    +    onExecutorRemoved(manager, "2")
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    assert(!executorsPendingToRemove(manager).contains("2"))
    +    assert(!executorsPendingToRemove(manager).contains("3"))
    +    onExecutorRemoved(manager, "2") // duplicates should not count
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    onExecutorRemoved(manager, "4")
    +    onExecutorRemoved(manager, "5")
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +
    +    // Try removing again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(!removeExecutor(manager, "7"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(!removeExecutor(manager, "8"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test ("interleaving add and remove") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Add a few executors
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 2)
    +    assert(addExecutors(manager) === 4)
    +    onExecutorAdded(manager, "1")
    +    onExecutorAdded(manager, "2")
    +    onExecutorAdded(manager, "3")
    +    onExecutorAdded(manager, "4")
    +    onExecutorAdded(manager, "5")
    +    onExecutorAdded(manager, "6")
    +    onExecutorAdded(manager, "7")
    +    assert(executorIds(manager).size === 7)
    +
    +    // Remove until limit
    +    assert(removeExecutor(manager, "1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(!removeExecutor(manager, "3")) // lower limit reached
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorRemoved(manager, "1")
    +    onExecutorRemoved(manager, "2")
    +    assert(executorIds(manager).size === 5)
    +
    +    // Add until limit
    +    assert(addExecutors(manager) === 5) // upper limit reached
    +    assert(addExecutors(manager) === 0)
    +    assert(!removeExecutor(manager, "3")) // still at lower limit
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorAdded(manager, "8")
    +    onExecutorAdded(manager, "9")
    +    onExecutorAdded(manager, "10")
    +    onExecutorAdded(manager, "11")
    +    onExecutorAdded(manager, "12")
    +    assert(executorIds(manager).size === 10)
    +
    +    // Remove succeeds again, now that we are no longer at the lower limit
    +    assert(removeExecutor(manager, "3"))
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(removeExecutor(manager, "6"))
    +    assert(executorIds(manager).size === 10)
    +    assert(addExecutors(manager) === 0) // still at upper limit
    +    onExecutorRemoved(manager, "3")
    +    onExecutorRemoved(manager, "4")
    +    assert(executorIds(manager).size === 8)
    +
    +    // Add succeeds again, now that we are no longer at the upper limit
    +    // Number of executors added restarts at 1
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    assert(executorIds(manager).size === 8)
    +    onExecutorRemoved(manager, "5")
    +    onExecutorRemoved(manager, "6")
    +    onExecutorAdded(manager, "13")
    +    onExecutorAdded(manager, "14")
    +    assert(executorIds(manager).size === 8)
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    onExecutorAdded(manager, "15")
    +    onExecutorAdded(manager, "16")
    +    assert(executorIds(manager).size === 10)
    +    sc.stop()
    +  }
    +
    +  test("starting/canceling add timer") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(8888L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting add timer is idempotent
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val firstAddTime = addTime(manager)
    +    assert(firstAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime) // timer is already started
    +    clock.tick(200L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime)
    +    onSchedulerQueueEmpty(manager)
    +
    +    // Restart add timer
    +    clock.tick(1000L)
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val secondAddTime = addTime(manager)
    +    assert(secondAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === secondAddTime) // timer is already started
    +    assert(addTime(manager) !== firstAddTime)
    +    assert(firstAddTime !== secondAddTime)
    +  }
    +
    +  test("starting/canceling remove timers") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(14444L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting remove timer is idempotent for each executor
    +    assert(removeTimes(manager).isEmpty)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("1"))
    +    val firstRemoveTime = removeTimes(manager)("1")
    +    assert(firstRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(100L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started
    +    clock.tick(200L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime)
    +    clock.tick(300L)
    +    onExecutorIdle(manager, "2")
    +    assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor
    +    assert(removeTimes(manager)("2") === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(400L)
    +    onExecutorIdle(manager, "3")
    +    assert(removeTimes(manager)("3") !== firstRemoveTime)
    +    assert(removeTimes(manager)("3") === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("2"))
    +    assert(removeTimes(manager).contains("3"))
    +
    +    // Restart remove timer
    +    clock.tick(1000L)
    +    onExecutorBusy(manager, "1")
    +    assert(removeTimes(manager).size === 2)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("1"))
    +    val secondRemoveTime = removeTimes(manager)("1")
    +    assert(secondRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started
    +    assert(removeTimes(manager)("1") !== firstRemoveTime)
    +    assert(firstRemoveTime !== secondRemoveTime)
    +  }
    +
    +  test("mock polling loop with no events") {
    +    val sc = createSparkContext(1, 20)
    +    val manager = sc.executorAllocationManager.get
    +    val clock = new TestClock(2020L)
    +    manager.setClock(clock)
    +
    +    // No events - we should not be adding or removing
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(100L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(1000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(10000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +  }
    +
    +  test("mock polling loop add behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Scheduler queue backlogged
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0) // timer not exceeded yet
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // first timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded
    +
    +    // Scheduler queue drained
    +    onSchedulerQueueEmpty(manager)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7) // timer is canceled
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7)
    +
    +    // Scheduler queue backlogged again
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1) // timer restarted
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 20) // limit reached
    +  }
    +
    +  test("mock polling loop remove behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Remove idle executors on timeout
    +    onExecutorAdded(manager, "executor-1")
    +    onExecutorAdded(manager, "executor-2")
    +    onExecutorAdded(manager, "executor-3")
    +    assert(removeTimes(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // idle threshold not reached yet
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle threshold exceeded
    +    assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining)
    +
    +    // Mark a subset as busy - only idle executors should be removed
    +    onExecutorAdded(manager, "executor-4")
    +    onExecutorAdded(manager, "executor-5")
    +    onExecutorAdded(manager, "executor-6")
    +    onExecutorAdded(manager, "executor-7")
    +    assert(removeTimes(manager).size === 5)              // 5 active executors
    +    assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed
    +    onExecutorBusy(manager, "executor-4")
    +    onExecutorBusy(manager, "executor-5")
    +    onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 2) // remove only idle executors
    +    assert(!removeTimes(manager).contains("executor-4"))
    +    assert(!removeTimes(manager).contains("executor-5"))
    +    assert(!removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle executors are removed
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("executor-4"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-5"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-6"))
    +
    +    // Busy executors are now idle and should be removed
    +    onExecutorIdle(manager, "executor-4")
    +    onExecutorIdle(manager, "executor-5")
    +    onExecutorIdle(manager, "executor-6")
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // 0 busy and 3 idle
    +    assert(removeTimes(manager).contains("executor-4"))
    +    assert(removeTimes(manager).contains("executor-5"))
    +    assert(removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty)
    +    assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining)
    +  }
    +
    +  test("listeners trigger add executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Starting a stage should start the add timer
    +    val numTasks = 10
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting a subset of the tasks should not cancel the add timer
    +    val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") }
    +    taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting all remaining tasks should cancel the add timer
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head))
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Start two different stages
    +    // The add timer should be canceled only if all tasks in both stages start running
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks)))
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) }
    +    assert(addTime(manager) === NOT_SET)
    +  }
    +
    +  test("listeners trigger remove executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // Added executors should start the remove timers for each executor
    +    (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) }
    +    assert(removeTimes(manager).size === 5)
    +
    +    // Starting a task cancel the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2")))
    +    assert(removeTimes(manager).size === 3)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +    assert(!removeTimes(manager).contains("executor-2"))
    +
    +    // Finishing all tasks running on an executor should start the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 4)
    +    assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet
    +    assert(removeTimes(manager).contains("executor-2"))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 5)
    +    assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished
    +  }
    +
    +  test("listeners trigger add and remove executor callbacks correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(executorIds(manager).isEmpty)
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // New executors have registered
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-1", "host1", 1), 100L))
    +    assert(executorIds(manager).size === 1)
    +    assert(executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("executor-1"))
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-2", "host2", 1), 100L))
    +    assert(executorIds(manager).size === 2)
    +    assert(executorIds(manager).contains("executor-2"))
    +    assert(removeTimes(manager).size === 2)
    +    assert(removeTimes(manager).contains("executor-2"))
    +
    +    // Existing executors have disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-1", "host1", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(!executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +
    +    // Unknown executor has disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-3", "host3", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(removeTimes(manager).size === 1)
    +  }
    +
    +}
    +
    +/**
    + * Helper methods for testing ExecutorAllocationManager.
    + * This includes methods to access private methods and fields in ExecutorAllocationManager.
    + */
    +private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
    +  private val schedulerBacklogTimeout = 1L
    +  private val sustainedSchedulerBacklogTimeout = 2L
    +  private val executorIdleTimeout = 3L
    +
    +  private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = {
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +      .set("spark.dynamicAllocation.minExecutors", minExecutors + "")
    +      .set("spark.dynamicAllocation.maxExecutors", maxExecutors + "")
    +      .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout",
    +        sustainedSchedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout + "")
    +    new SparkContext(conf)
    +  }
    +
    +  private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = {
    +    new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details")
    +  }
    +
    +  private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = {
    +    new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false)
    +  }
    +
    +  /* ------------------------------------------------------- *
    +   | Helper methods for accessing private methods and fields |
    --- End diff --
    
    Yeah, you annotate the original method. I think it's an ok compromise when you want to test APIs that really should be private - open up their visibility just enough and tack on the annotation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58722547
  
    This is looking good @andrewor14 .  Posted a few comments inline and had a broader question:
    
    Would you mind motivating the Timer machinery a little bit?  It took me a while to understand how to reason about, and I'm wondering whether a simpler approach would make sense.  IIUC, it also means that every time we submit a task against an idle cluster, even if it gets dispatched quickly, we end up scheduling and then cancelling timer tasks.  Would it make sense to have noMorePendingTasks and newPendingTasks simply update a variable that holds the last time we transitioned from no pending tasks to pending tasks? And then to have a single timer that checks this variable periodically and makes decisions based on it?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377840
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd *= 2
    +      numExecutorsPending += actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are " +
    +        s"only $minNumExecutors executor(s) left, which is the limit")
    +      return
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be triggered
    +   * in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in every round
    +   * thereafter. This is called when the scheduler receives new pending tasks.
    +   */
    +  def startAddTimer(): Unit = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    +    logDebug(s"Canceling remove executor timer for $executorId")
    +    removeTimes.remove(executorId)
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  private val NOT_STARTED = -1L
    +}
    +
    +/**
    + * A listener that notifies the given allocation manager of when to add and remove executors.
    + */
    +private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +  extends SparkListener {
    +
    +  private val stageIdToPendingTaskIndex = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +  private val executorIdToTaskId = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
    +    val stageId = stageSubmitted.stageInfo.stageId
    +    val numTasks = stageSubmitted.stageInfo.numTasks
    +    // Start the add timer because there are new pending tasks
    +    stageIdToPendingTaskIndex.getOrElseUpdate(
    +      stageId, new mutable.HashSet[Int]) ++= (0 to numTasks - 1)
    --- End diff --
    
    I don't think it's correct here to assume that if a stage has N tasks that it has tasks of index (0 to N-1). In some cases a stage can be submitted that computes only some indices of an RDD. I think in this case it is sufficient to just keep track of numTasks for each stage and then keep track of the number of distinct taskIndices that have completed. Once the size of that set == numTasks, you can assume it's fine. One other question is how this works with speculation... it would be good to look at the code and understand how speculative tasks are submitted to the listener (I think right now there is no way for the listener to know about them) /cc @kayousterhout.
    
    It might be fine to simply ignore these for now when computing the backlog. However, we need to make sure there is no corner case where a task is submitted according to this Listener but we never clear it. Otherwise we might constantly think we are backlogged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60040691
  
    Hey Andrew - I looked at this and I had some small suggestions around naming.
    
    However, there is a big open question here relating to the semantics of requesting more resources from things like YARN and standalone (/cc @kayousterhout @sryza @vanzin) and we need to define that API precisely before we can know what to do here.
    
    The code here uses timeouts in order to prevent requesting too many executors. But this doesn't seem correct to me. What happens if this code "times out" a request after some time, but actually the YARN scheduler still has that request and plans to fullfil it later once resources are available. In that case, you could over allocate because you might send another request later and YARN will actually fulfill both requests.
    
    There are different levels of robustness we could expect from the scheduler API's @sryza @vanzin might have some insight for YARN:
    
    1. _Guaranteed instant fulfillment_ If I request N executors I will get them instantly.
    2. _Guaranteed eventual fullfilment_ If I request N executors I will get them eventually, provided that enough resources become available in my YARN queue.
    3. _Best effort with acknowledgment_ If I request N executors, the request might not be fulfilled, even if sufficient resources eventually become available in my YARN queue. However, there is a way to know when requests are "dropped" - i.e. when they will no longer be considered for fulfillment.
    4. _Best effort_ If I request N executors, the request might not be fulfilled, even if sufficient resources eventually become available in my YARN queue. YARN provides no way to track whether a given resource request is going to be fulfilled or not.
    
    (1) is impossible to fulfill since someone could e.g. request more resources than the question.
    
    I'm actually not sure whether YARN offers (2) (3) or (4). It would be good to know that. The only case where I think it makes sense to have our own timeouts is (4) since there is really nothing we can do. If YARN provides (2), then we should just assume that if requests are pending, there is nothing more we can do - that would be the simplest. If YARN supports (3) it's a bit trickier.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922896
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    --- End diff --
    
    Yeah, my original thinking is that if we're hitting this case then we're adding executors too quickly so we need to slow down, but in retrospect that might be too conservative. For instance, if we set the interval to a really low value then we'll basically always hit this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60823760
  
    @pwendell yeah, that looks a lot easier to reason about. Also, I'd point out that if following that suggestion, the allocation code should try to start `--min-executors` right away instead of doing a slow start to reach the minimum.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59304443
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21787/consoleFull) for   PR 2746 at commit [`44f1832`](https://github.com/apache/spark/commit/44f1832e5be49a7c6f215a318e18c4721650a075).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18931377
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logInfo(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    // Start the retry timer in case this addition fails
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total is $newTotalExecutors)")
    +    numExecutorsToAdd *= 2
    +    numExecutorsPendingToAdd += actualNumExecutorsToAdd
    +    backend.requestExecutors(actualNumExecutorsToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Retry a previous executor request that has not been fulfilled.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryAddExecutors(): Unit = synchronized {
    +    // Do not retry if there are no executors pending to be added (should never happen)
    +    if (numExecutorsPendingToAdd == 0) {
    +      logWarning("Attempted to retry adding executors when there are none pending to be added")
    +      cancelAddRetryTimer()
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    addRetryAttempts += 1
    +    if (addRetryAttempts > maxAddRetryAttempts) {
    +      logInfo(s"Giving up on adding $numExecutorsPendingToAdd executor(s) " +
    +        s"after $maxAddRetryAttempts failed attempts")
    +      numExecutorsPendingToAdd = 0
    +      // Also cancel original add timer because the cluster is not granting us new executors
    +      cancelAddTimer()
    +      return
    +    }
    +
    +    // Retry a previous request, then restart the retry timer in case this retry also fails
    +    logInfo(s"Previously requested executors have not all registered yet. " +
    +      s"Retrying to add $numExecutorsPendingToAdd executor(s) " +
    +      s"[attempt $addRetryAttempts/$maxAddRetryAttempts]")
    +    backend.requestExecutors(numExecutorsPendingToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Request the scheduler backend to decommission the given executor.
    +   * This expires the remove timer unless the executor is kept alive intentionally.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId) ||
    +        removeRetryAttempts.contains(executorId) ||
    +        retryRemoveTimers.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit")
    +      // Restart the remove timer to keep the executor marked as idle
    +      // Otherwise we won't be able to remove this executor even after new executors have joined
    +      startRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Send a kill request to the backend for this executor
    +    // Start the retry timer in case this removal fails
    +    logInfo(s"Removing executor $executorId because it has been idle for " +
    +      s"$removeThreshold seconds (new total is ${numExistingExecutors - 1})")
    +    executorsPendingToRemove.add(executorId)
    +    backend.killExecutor(executorId)
    +    cancelRemoveTimer(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Retry a previous attempt to decommission the given executor.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryRemoveExecutors(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to retry removing unknown executor $executorId")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if this executor is not pending to be killed (should never happen)
    +    if (!executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Attempted to retry removing executor $executorId when it's not to be removed!")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    removeRetryAttempts(executorId) =
    +      removeRetryAttempts.getOrElse(executorId, 0) + 1
    +    if (removeRetryAttempts(executorId) > maxRemoveRetryAttempts) {
    +      logInfo(s"Giving up on removing executor $executorId after " +
    +        s"$maxRemoveRetryAttempts failed attempts!")
    +      executorsPendingToRemove.remove(executorId)
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Retry a previous kill request for this executor
    +    // Restart the retry timer in case this retry also fails
    +    logInfo(s"Retrying previous attempt to remove $executorId " +
    +      s"[attempt ${removeRetryAttempts(executorId)}/$maxRemoveRetryAttempts]")
    +    backend.killExecutor(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPendingToAdd left)")
    +        if (numExecutorsPendingToAdd == 0) {
    +          logDebug("All previously pending executors have registered!")
    +          cancelAddRetryTimer()
    +        }
    +      }
    +      executorIds.add(executorId)
    +      startRemoveTimer(executorId)
    --- End diff --
    
    Nevermind, I think I get it after reading the scheduler code. Nevertheless, it would be nice to explain this in the comment ("the executor is scheduled for removal right away, and its lifetime is prolonged as tasks are scheduled to run on it" or something).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60708941
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22326/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60475508
  
    The latest set of commits replaced all the new logic in the scheduler package with a listener. This isolates the new logic very nicely within the new file as @kayousterhout suggested. Note that I haven't addressed all review comments yet. I plan to do that and write tests shortly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18920907
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    --- End diff --
    
    Is it possible to expose this information somewhere in a more general way?  It seems a little hacky to duplicate tracking here and in the scheduler code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59562692
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21859/consoleFull) for   PR 2746 at commit [`9d516cc`](https://github.com/apache/spark/commit/9d516ccdcef852644a5a3423ad7f6dd2f5820dad).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60343192
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22105/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58609625
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21568/consoleFull) for   PR 2746 at commit [`67c03c7`](https://github.com/apache/spark/commit/67c03c7ca4781d62da41b995e59014c374931e99).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59143124
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21750/consoleFull) for   PR 2746 at commit [`baaa403`](https://github.com/apache/spark/commit/baaa403b16d10107abeeb1fba721b2c301d2c0aa).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18732982
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors
    +  private val addExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60) // s
    +  private val addExecutorInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addExecutorThreshold) // s
    +  private val removeExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 300) // s
    +
    +  // Timers that keep track of when to add and remove executors
    +  private var addExecutorTimer: Option[Timer] = None
    +  private val removeExecutorTimers: mutable.Map[String, Timer] = new mutable.HashMap[String, Timer]
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // The number of pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private var numExecutorsPendingToRemove = 0
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String] ++= scheduler.executorIdToHost.keys
    +
    +  // Start idle timer for all new executors
    +  synchronized { executorIds.foreach(startRemoveExecutorTimer) }
    +
    +  /**
    +   * Start the add executor timer if it does not already exist.
    +   * This is called when a new pending task is added. The add is then triggered
    +   * if the pending tasks queue is not drained in `addExecutorThreshold` seconds.
    +   */
    +  def startAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorThreshold)
    +
    +  /**
    +   * Restart the add executor timer.
    +   * This is called when the previous add executor timer has expired but not canceled. The add
    +   * is then triggered again if all pending executors from the previous round have registered,
    +   * and the pending tasks queue is still not drained in `addExecutorInterval` seconds.
    +   */
    +  private def restartAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorInterval)
    +
    +  /**
    +   * Start the add executor timer using the given delay if the timer does not already exist.
    +   */
    +  private def startAddExecutorTimer(timerDelaySeconds: Long): Unit = {
    +    addExecutorTimer.synchronized {
    +      if (addExecutorTimer.isEmpty) {
    +        logDebug(s"Starting add executor timer (to expire in $timerDelaySeconds seconds)")
    +        addExecutorTimer = Some(new Timer)
    +        addExecutorTimer.get.schedule(
    +          new AddExecutorTimerTask(numExecutorsToAdd), timerDelaySeconds * 1000)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor if the timer does not already exist.
    +   * This is called when the executor initially registers with the driver or finishes running
    +   * a task. The removal is then triggered if the executor stays idle (i.e. not running a task)
    +   * for `removeExecutorThreshold` seconds.
    +   */
    +  def startRemoveExecutorTimer(executorId: String): Unit = {
    +    removeExecutorTimers.synchronized {
    +      if (!removeExecutorTimers.contains(executorId)) {
    +        logDebug(s"Starting idle timer for executor $executorId " +
    +          s"(to expire in $removeExecutorThreshold seconds)")
    +        removeExecutorTimers(executorId) = new Timer
    +        removeExecutorTimers(executorId).schedule(
    +          new RemoveExecutorTimerTask(executorId), removeExecutorThreshold * 1000)
    +      }
    +    }
    +    // Acquire a more general lock here because we might mutate `executorId`
    +    synchronized {
    +      if (!executorIds.contains(executorId)) {
    +        logWarning(s"Started idle timer for unknown executor $executorId.")
    +        executorIds.add(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that adds executors.
    +   * This is called when the pending task queue is drained.
    +   */
    +  def cancelAddExecutorTimer(): Unit = addExecutorTimer.synchronized {
    +    addExecutorTimer.foreach { timer =>
    +      logDebug("Canceling add executor timer because task queue is drained!")
    +      timer.cancel()
    +      numExecutorsToAdd = 1
    +      addExecutorTimer = None
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that removes the given executor.
    +   * This is called when the executor is no longer idle.
    +   */
    +  def cancelRemoveExecutorTimer(executorId: String): Unit = removeExecutorTimers.synchronized {
    +    if (removeExecutorTimers.contains(executorId)) {
    +      logDebug(s"Canceling idle timer for executor $executorId.")
    +      removeExecutorTimers(executorId).cancel()
    +      removeExecutorTimers.remove(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Negotiate with the scheduler backend to add new executors.
    +   * This ensures the resulting number of executors is correctly constrained by the upper bound.
    +   * Return the number of executors actually requested.
    +   */
    +  private def addExecutors(numExecutorsRequested: Int): Int = synchronized {
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    val numExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsRequested <= maxNumExecutors) {
    +        numExecutorsRequested
    +      } else {
    +        // Add just enough to reach `maxNumExecutors`
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newNumExecutors = numExistingExecutors + numExecutorsToAdd
    +
    +    if (numExecutorsToAdd > 0) {
    +      getCoarseGrainedBackend.foreach { backend =>
    +        logInfo(s"Pending tasks are building up! " +
    +          s"Adding $numExecutorsToAdd new executor(s) (new total is $newNumExecutors).")
    --- End diff --
    
    It would be good to be clear that the executors aren't actually added yet.  Maybe "Requesting... (new desired total is ...)"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18934791
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    --- End diff --
    
    It's not. Actually on second thought this doesn't really do anything, because at this point we haven't even started the executors yet. Also, we get the most refreshed list of executors every second or so through `TaskSchedulerImpl#resourceOffers`, so I think it's safe to remove this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59143491
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21753/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19561086
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -0,0 +1,661 @@
    +/*
    + * 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
    +
    +import org.scalatest.{FunSuite, PrivateMethodTester}
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.scheduler._
    +import org.apache.spark.storage.BlockManagerId
    +
    +/**
    + * Test add and remove behavior of ExecutorAllocationManager.
    + */
    +class ExecutorAllocationManagerSuite extends FunSuite {
    +  import ExecutorAllocationManager._
    +  import ExecutorAllocationManagerSuite._
    +
    +  test("verify min/max executors") {
    +    // No min or max
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +    intercept[SparkException] { new SparkContext(conf) }
    +
    +    // Only min
    +    val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
    +    intercept[SparkException] { new SparkContext(conf1) }
    +
    +    // Only max
    +    val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
    +    intercept[SparkException] { new SparkContext(conf2) }
    +
    +    // Both min and max, but min > max
    +    intercept[SparkException] { createSparkContext(2, 1) }
    +
    +    // Both min and max, and min == max
    +    val sc1 = createSparkContext(1, 1)
    +    assert(sc1.executorAllocationManager.isDefined)
    +    sc1.stop()
    +
    +    // Both min and max, and min < max
    +    val sc2 = createSparkContext(1, 2)
    +    assert(sc2.executorAllocationManager.isDefined)
    +    sc2.stop()
    +  }
    +
    +  test("starting state") {
    +    val sc = createSparkContext()
    +    val manager = sc.executorAllocationManager.get
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(executorIds(manager).isEmpty)
    +    assert(addTime(manager) === ExecutorAllocationManager.NOT_SET)
    +    assert(removeTimes(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test("add executors") {
    +    val sc = createSparkContext(1, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Keep adding until the limit is reached
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 1)
    +    assert(numExecutorsPending(manager) === 1)
    +    assert(numExecutorsToAdd(manager) === 2)
    +    assert(addExecutors(manager) === 2)
    +    assert(numExecutorsPending(manager) === 3)
    +    assert(numExecutorsToAdd(manager) === 4)
    +    assert(addExecutors(manager) === 4)
    +    assert(numExecutorsPending(manager) === 7)
    +    assert(numExecutorsToAdd(manager) === 8)
    +    assert(addExecutors(manager) === 3) // reached the limit of 10
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +
    +    // Register previously requested executors
    +    onExecutorAdded(manager, "first")
    +    assert(numExecutorsPending(manager) === 9)
    +    onExecutorAdded(manager, "second")
    +    onExecutorAdded(manager, "third")
    +    onExecutorAdded(manager, "fourth")
    +    assert(numExecutorsPending(manager) === 6)
    +    onExecutorAdded(manager, "first") // duplicates should not count
    +    onExecutorAdded(manager, "second")
    +    assert(numExecutorsPending(manager) === 6)
    +
    +    // Try adding again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    sc.stop()
    +  }
    +
    +  test("remove executors") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +    (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) }
    +
    +    // Keep removing until the limit is reached
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(removeExecutor(manager, "1"))
    +    assert(executorsPendingToRemove(manager).size === 1)
    +    assert(executorsPendingToRemove(manager).contains("1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(removeExecutor(manager, "3"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).contains("2"))
    +    assert(executorsPendingToRemove(manager).contains("3"))
    +    assert(!removeExecutor(manager, "100")) // remove non-existent executors
    +    assert(!removeExecutor(manager, "101"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(!removeExecutor(manager, "6")) // reached the limit of 5
    +    assert(executorsPendingToRemove(manager).size === 5)
    +    assert(executorsPendingToRemove(manager).contains("4"))
    +    assert(executorsPendingToRemove(manager).contains("5"))
    +    assert(!executorsPendingToRemove(manager).contains("6"))
    +
    +    // Kill executors previously requested to remove
    +    onExecutorRemoved(manager, "1")
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("1"))
    +    onExecutorRemoved(manager, "2")
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    assert(!executorsPendingToRemove(manager).contains("2"))
    +    assert(!executorsPendingToRemove(manager).contains("3"))
    +    onExecutorRemoved(manager, "2") // duplicates should not count
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    onExecutorRemoved(manager, "4")
    +    onExecutorRemoved(manager, "5")
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +
    +    // Try removing again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(!removeExecutor(manager, "7"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(!removeExecutor(manager, "8"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test ("interleaving add and remove") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Add a few executors
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 2)
    +    assert(addExecutors(manager) === 4)
    +    onExecutorAdded(manager, "1")
    +    onExecutorAdded(manager, "2")
    +    onExecutorAdded(manager, "3")
    +    onExecutorAdded(manager, "4")
    +    onExecutorAdded(manager, "5")
    +    onExecutorAdded(manager, "6")
    +    onExecutorAdded(manager, "7")
    +    assert(executorIds(manager).size === 7)
    +
    +    // Remove until limit
    +    assert(removeExecutor(manager, "1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(!removeExecutor(manager, "3")) // lower limit reached
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorRemoved(manager, "1")
    +    onExecutorRemoved(manager, "2")
    +    assert(executorIds(manager).size === 5)
    +
    +    // Add until limit
    +    assert(addExecutors(manager) === 5) // upper limit reached
    +    assert(addExecutors(manager) === 0)
    +    assert(!removeExecutor(manager, "3")) // still at lower limit
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorAdded(manager, "8")
    +    onExecutorAdded(manager, "9")
    +    onExecutorAdded(manager, "10")
    +    onExecutorAdded(manager, "11")
    +    onExecutorAdded(manager, "12")
    +    assert(executorIds(manager).size === 10)
    +
    +    // Remove succeeds again, now that we are no longer at the lower limit
    +    assert(removeExecutor(manager, "3"))
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(removeExecutor(manager, "6"))
    +    assert(executorIds(manager).size === 10)
    +    assert(addExecutors(manager) === 0) // still at upper limit
    +    onExecutorRemoved(manager, "3")
    +    onExecutorRemoved(manager, "4")
    +    assert(executorIds(manager).size === 8)
    +
    +    // Add succeeds again, now that we are no longer at the upper limit
    +    // Number of executors added restarts at 1
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    assert(executorIds(manager).size === 8)
    +    onExecutorRemoved(manager, "5")
    +    onExecutorRemoved(manager, "6")
    +    onExecutorAdded(manager, "13")
    +    onExecutorAdded(manager, "14")
    +    assert(executorIds(manager).size === 8)
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    onExecutorAdded(manager, "15")
    +    onExecutorAdded(manager, "16")
    +    assert(executorIds(manager).size === 10)
    +    sc.stop()
    +  }
    +
    +  test("starting/canceling add timer") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(8888L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting add timer is idempotent
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val firstAddTime = addTime(manager)
    +    assert(firstAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime) // timer is already started
    +    clock.tick(200L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime)
    +    onSchedulerQueueEmpty(manager)
    +
    +    // Restart add timer
    +    clock.tick(1000L)
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val secondAddTime = addTime(manager)
    +    assert(secondAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === secondAddTime) // timer is already started
    +    assert(addTime(manager) !== firstAddTime)
    +    assert(firstAddTime !== secondAddTime)
    +  }
    +
    +  test("starting/canceling remove timers") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(14444L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting remove timer is idempotent for each executor
    +    assert(removeTimes(manager).isEmpty)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("1"))
    +    val firstRemoveTime = removeTimes(manager)("1")
    +    assert(firstRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(100L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started
    +    clock.tick(200L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime)
    +    clock.tick(300L)
    +    onExecutorIdle(manager, "2")
    +    assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor
    +    assert(removeTimes(manager)("2") === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(400L)
    +    onExecutorIdle(manager, "3")
    +    assert(removeTimes(manager)("3") !== firstRemoveTime)
    +    assert(removeTimes(manager)("3") === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("2"))
    +    assert(removeTimes(manager).contains("3"))
    +
    +    // Restart remove timer
    +    clock.tick(1000L)
    +    onExecutorBusy(manager, "1")
    +    assert(removeTimes(manager).size === 2)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("1"))
    +    val secondRemoveTime = removeTimes(manager)("1")
    +    assert(secondRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started
    +    assert(removeTimes(manager)("1") !== firstRemoveTime)
    +    assert(firstRemoveTime !== secondRemoveTime)
    +  }
    +
    +  test("mock polling loop with no events") {
    +    val sc = createSparkContext(1, 20)
    +    val manager = sc.executorAllocationManager.get
    +    val clock = new TestClock(2020L)
    +    manager.setClock(clock)
    +
    +    // No events - we should not be adding or removing
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(100L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(1000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(10000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +  }
    +
    +  test("mock polling loop add behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Scheduler queue backlogged
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0) // timer not exceeded yet
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // first timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded
    +
    +    // Scheduler queue drained
    +    onSchedulerQueueEmpty(manager)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7) // timer is canceled
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7)
    +
    +    // Scheduler queue backlogged again
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1) // timer restarted
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 20) // limit reached
    +  }
    +
    +  test("mock polling loop remove behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Remove idle executors on timeout
    +    onExecutorAdded(manager, "executor-1")
    +    onExecutorAdded(manager, "executor-2")
    +    onExecutorAdded(manager, "executor-3")
    +    assert(removeTimes(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // idle threshold not reached yet
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle threshold exceeded
    +    assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining)
    +
    +    // Mark a subset as busy - only idle executors should be removed
    +    onExecutorAdded(manager, "executor-4")
    +    onExecutorAdded(manager, "executor-5")
    +    onExecutorAdded(manager, "executor-6")
    +    onExecutorAdded(manager, "executor-7")
    +    assert(removeTimes(manager).size === 5)              // 5 active executors
    +    assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed
    +    onExecutorBusy(manager, "executor-4")
    +    onExecutorBusy(manager, "executor-5")
    +    onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 2) // remove only idle executors
    +    assert(!removeTimes(manager).contains("executor-4"))
    +    assert(!removeTimes(manager).contains("executor-5"))
    +    assert(!removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle executors are removed
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("executor-4"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-5"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-6"))
    +
    +    // Busy executors are now idle and should be removed
    +    onExecutorIdle(manager, "executor-4")
    +    onExecutorIdle(manager, "executor-5")
    +    onExecutorIdle(manager, "executor-6")
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // 0 busy and 3 idle
    +    assert(removeTimes(manager).contains("executor-4"))
    +    assert(removeTimes(manager).contains("executor-5"))
    +    assert(removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty)
    +    assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining)
    +  }
    +
    +  test("listeners trigger add executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Starting a stage should start the add timer
    +    val numTasks = 10
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting a subset of the tasks should not cancel the add timer
    +    val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") }
    +    taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting all remaining tasks should cancel the add timer
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head))
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Start two different stages
    +    // The add timer should be canceled only if all tasks in both stages start running
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks)))
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) }
    +    assert(addTime(manager) === NOT_SET)
    +  }
    +
    +  test("listeners trigger remove executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // Added executors should start the remove timers for each executor
    +    (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) }
    +    assert(removeTimes(manager).size === 5)
    +
    +    // Starting a task cancel the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2")))
    +    assert(removeTimes(manager).size === 3)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +    assert(!removeTimes(manager).contains("executor-2"))
    +
    +    // Finishing all tasks running on an executor should start the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 4)
    +    assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet
    +    assert(removeTimes(manager).contains("executor-2"))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 5)
    +    assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished
    +  }
    +
    +  test("listeners trigger add and remove executor callbacks correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(executorIds(manager).isEmpty)
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // New executors have registered
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-1", "host1", 1), 100L))
    +    assert(executorIds(manager).size === 1)
    +    assert(executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("executor-1"))
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-2", "host2", 1), 100L))
    +    assert(executorIds(manager).size === 2)
    +    assert(executorIds(manager).contains("executor-2"))
    +    assert(removeTimes(manager).size === 2)
    +    assert(removeTimes(manager).contains("executor-2"))
    +
    +    // Existing executors have disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-1", "host1", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(!executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +
    +    // Unknown executor has disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-3", "host3", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(removeTimes(manager).size === 1)
    +  }
    +
    +}
    +
    +/**
    + * Helper methods for testing ExecutorAllocationManager.
    + * This includes methods to access private methods and fields in ExecutorAllocationManager.
    + */
    +private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
    +  private val schedulerBacklogTimeout = 1L
    +  private val sustainedSchedulerBacklogTimeout = 2L
    +  private val executorIdleTimeout = 3L
    +
    +  private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = {
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +      .set("spark.dynamicAllocation.minExecutors", minExecutors + "")
    +      .set("spark.dynamicAllocation.maxExecutors", maxExecutors + "")
    +      .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout",
    +        sustainedSchedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout + "")
    +    new SparkContext(conf)
    +  }
    +
    +  private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = {
    +    new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details")
    +  }
    +
    +  private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = {
    +    new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false)
    +  }
    +
    +  /* ------------------------------------------------------- *
    +   | Helper methods for accessing private methods and fields |
    --- End diff --
    
    Yeah I think I've seen that in the hadoop code. If I understand correctly, that requires adding the annotation to the original method, not the test method, right? An advantage here is that the original code doesn't need to change in anyway and we can achieve the same. Unfortunately the API exposed here is quite verbose.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60036614
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22020/consoleFull) for   PR 2746 at commit [`8bc0e9d`](https://github.com/apache/spark/commit/8bc0e9d939c9e306ae46c60483e93eda992de1e6).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60715271
  
    @andrewor14 this is looking good. All my comments are about some refactoring to make it more testable, and dealing with error conditions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60547159
  
      [Test build #22271 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22271/consoleFull) for   PR 2746 at commit [`254c958`](https://github.com/apache/spark/commit/254c9584d67f674efe828e07ff6c4222e0386c4d).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58606196
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21568/consoleFull) for   PR 2746 at commit [`67c03c7`](https://github.com/apache/spark/commit/67c03c7ca4781d62da41b995e59014c374931e99).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19194567
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    --- End diff --
    
    Using the word "timer" here doesn't convey much information. What about saying:
    
    ```
    If set, this value indicates the absolute time after which we should start adding executors.
    It is updated in real time based on the current state of the task queue.
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60863405
  
      [Test build #22416 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22416/consoleFull) for   PR 2746 at commit [`b00b680`](https://github.com/apache/spark/commit/b00b680639fbb73778fe2eb28ec54330c99b9d72).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59854068
  
    @sryza What's the policy used by MR/Tez?  When talking to Patrick/Andrew offline, I'd argued for allocating executors such that the total executors given to the driver is equal to the (# pending tasks) / (cores per executor) (subject, of course, to fairness constraints that might limit this to be less), which I thought might provide better "out of the box" behavior without needing to set config parameters.  This approach was vetoed by others as too hard to understand.  So, I'm curious what MR/Tez do and whether their approach is perceived as easy to understand?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58759844
  
    Awesome, sounds good, will hold off.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59304965
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21786/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61022171
  
      [Test build #22487 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22487/consoleFull) for   PR 2746 at commit [`e7f9684`](https://github.com/apache/spark/commit/e7f9684ec934a00b98fc1d72abd15fbd859ce8e1).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922597
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -110,19 +110,26 @@ private[spark] class TaskSetManager(
       // the one that it was launched from, but gets removed from them later.
       private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
    +  // List 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.
    +  // List of pending tasks for each rack -- similar to the above.
       private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set containing pending tasks with no locality preferences.
    +  // List of pending tasks with no locality preferences.
       var pendingTasksWithNoPrefs = new ArrayBuffer[Int]
     
    -  // Set containing all pending tasks (also used as a stack, as above).
    +  // List of all pending tasks (also used as a stack, as above).
       val allPendingTasks = new ArrayBuffer[Int]
     
    +  // Set of pending tasks used to keep track of whether more executors are needed
    --- End diff --
    
    Actually I don't understand why we can't just combine this will allPendingTasks?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60708935
  
      [Test build #22326 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22326/consoleFull) for   PR 2746 at commit [`abdea61`](https://github.com/apache/spark/commit/abdea61a7d797a822b1cdc539da964134f546ac9).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61025246
  
      [Test build #22485 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22485/consoleFull) for   PR 2746 at commit [`340f3b1`](https://github.com/apache/spark/commit/340f3b19184b9709020306a84872068a5d5ea507).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60869230
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22416/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60874822
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22422/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19515974
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    --- End diff --
    
    Actually, given that this is likely not the last patch of this bigger feature, I will leave the documentation for later. This feature is technically not ready until the off-heap shuffle service is also merged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60040481
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22020/consoleFull) for   PR 2746 at commit [`8bc0e9d`](https://github.com/apache/spark/commit/8bc0e9d939c9e306ae46c60483e93eda992de1e6).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59138015
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21747/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59133473
  
    I have updated the code to use polling rather than Java `Timers`. The new approach uses only 1 extra thread instead of 1 for each executor, and I find the logic simpler to implement and reason about.
    
    @sryza This still uses the exponential increase policy where each round stops and wait until the executors are actually granted by the cluster manager before continuing. I understand this does not address the concerns you raised on the corresponding JIRA yet, so I am working out the proper semantics there myself. Other than that, feel free to review the rest of the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18920574
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    --- End diff --
    
    The number of configuration options here is a little scary.  Do we need all of these things to be configurable?  The first 3 seem like things we definitely need.  Some of the remaining things seem very tied to how long it takes a new executor to start up, and I wonder if we could hardcode them for now, and make them configurable later only if people find the defaults to be problematic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59280725
  
    @andrewor14 this looks cool!  Two high level comments:
    
    (1) Right now, there are a bunch of different kinds of timers that can be set and then triggered later.  Is it possible to do something much simpler, where you just have a single timer, and when the timer goes off, you check all of the state to see what should happen next? For example, for executor removal, you could have a map of executors to when they were last used (or sorted list based on when they were last used?) and remove any that haven't been used for a threshold amount of time.  As I read the code, I was wondering if this would significantly simplify a bunch of the code, with negligible performance impact (?).
    
    (2) I'm a little concerned about the changes to the scheduler code.  In general, the scheduler has evolved to this point where the interface between the various scheduler components is very messy, and it's really hard to describe (does anyone even know?!) what happens in which part of the scheduler.  The way you've done this right now adds some logic in the task set manager (to determine when something has changed on the level of the task set, in which case the task scheduler needs to be notified) and to the task scheduler implementation, to determine when something has changed across all task sets, in which case it needs to notify the Allocation Manager.
    
    Based on a cursory look, it looks like potentially *all* of the logic added to the task scheduler impl could instead be done in the Allocation Manager (by moving all of the data structures you added in lines 118 - 132 to the Allocation Manager).  I'd prefer this because then the Allocation Manager can have a very narrow interface that shields the scheduler from a lot of complexity.
    
    In this case, you could pass a reference to Allocation Manager to each TSM, which would call a "newPendingTask" method on the Allocation Manager (for example).  Similar to with the TaskSchedulerImpl, I think most of the functionality in the TSM related to the allocation manager code could be pushed into the allocation manager, so that the TSM doesn't have to handle any of the complexity of tracking new information itself -- instead, it would just call a method on the Allocation Manager when tasks get started / finish.
    
    One potential issue I see with pushing all of the functionality from the TSM into the allocation manager is that right now, the TSM can drop some messages from getting sent back through the scheduler / allocation manager -- like when a task finishes on an executor, but there are other tasks for the task set still running there, so the allocation manager doesn't need to know about it.  I'd guess this isn't a huge performance hit -- the complexity of all of this logic is still O(number of tasks being scheduled) and is merited by the simplicity benefit of having the AllocationManager do everything.
    
    At the end of the day, it seems like the AllocationManager needs to know when a task starts and when one finishes.  So, can it just implement the listener interface, and the scheduler doesn't directly interact with it at all?  It looks like you mentioned this idea in your design doc, so I'm wondering if you stumbled upon some problem when you tried doing this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19432659
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    --- End diff --
    
    Yeah, especially for yarn-client mode because the driver is not co-located with the AM, but it's also needed for yarn-cluster mode because the request might be dropped (e.g. akka's `!` is fire-and-forget).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922282
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -110,19 +110,26 @@ private[spark] class TaskSetManager(
       // the one that it was launched from, but gets removed from them later.
       private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
    +  // List 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.
    +  // List of pending tasks for each rack -- similar to the above.
       private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set containing pending tasks with no locality preferences.
    +  // List of pending tasks with no locality preferences.
       var pendingTasksWithNoPrefs = new ArrayBuffer[Int]
     
    -  // Set containing all pending tasks (also used as a stack, as above).
    +  // List of all pending tasks (also used as a stack, as above).
       val allPendingTasks = new ArrayBuffer[Int]
     
    +  // Set of pending tasks used to keep track of whether more executors are needed
    --- End diff --
    
    Describe how this is different than allPendingTasks in the comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60549934
  
      [Test build #22272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22272/consoleFull) for   PR 2746 at commit [`149cc32`](https://github.com/apache/spark/commit/149cc32bee25b515da5a38edc2659af8aaaff2d2).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60862786
  
      [Test build #22412 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22412/consoleFull) for   PR 2746 at commit [`c3caa65`](https://github.com/apache/spark/commit/c3caa650108edde0b1386098368ad73e49bb3834).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60705736
  
    Alright, I think I have addressed everyone's comments at this point. If I missed something, please let me know. The only thing left to do now from my side is documenting the configs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60837794
  
    Would it make sense to start it at `--min-executors` or `--max-executors`? I would imagine the application  wanting to start out with many executors from the beginning to run the first job.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60870010
  
      [Test build #22422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22422/consoleFull) for   PR 2746 at commit [`c60ec33`](https://github.com/apache/spark/commit/c60ec33219a9341fe5c3a7c25c148b37940ba1e3).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19384336
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd *= 2
    +      numExecutorsPending += actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are " +
    +        s"only $minNumExecutors executor(s) left, which is the limit")
    +      return
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be triggered
    +   * in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in every round
    +   * thereafter. This is called when the scheduler receives new pending tasks.
    +   */
    +  def startAddTimer(): Unit = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    +    logDebug(s"Canceling remove executor timer for $executorId")
    +    removeTimes.remove(executorId)
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  private val NOT_STARTED = -1L
    +}
    +
    +/**
    + * A listener that notifies the given allocation manager of when to add and remove executors.
    + */
    +private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +  extends SparkListener {
    +
    +  private val stageIdToPendingTaskIndex = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +  private val executorIdToTaskId = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
    +    val stageId = stageSubmitted.stageInfo.stageId
    +    val numTasks = stageSubmitted.stageInfo.numTasks
    +    // Start the add timer because there are new pending tasks
    +    stageIdToPendingTaskIndex.getOrElseUpdate(
    +      stageId, new mutable.HashSet[Int]) ++= (0 to numTasks - 1)
    --- End diff --
    
    Another thing to think about here is cancellations - what happens if a stage is started but it's cancelled before all of its tasks complete.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61029104
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22487/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19389557
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    --- End diff --
    
    For the way apps normally use YARN, the concept of "request acknowledged" doesn't really make sense.  Updating the local bookkeeping on the number of executors needed is usually asynchonous from passing on that requirement to the ResourceManager.  YarnAllocator should have a thread that periodically heartbeats to the ResourceManager.  If there's a temporary network partition that happens to coincide with the addTimer expiring, we'll communicate what we needed to on the next heartbeat.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19195340
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  def initialize(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +
    +              // Expire any outstanding pending add requests that have timed out
    +              if (pendingAddExpirationTime != NOT_STARTED && now >= pendingAddExpirationTime) {
    +                logDebug(s"Expiring all pending add requests because they have " +
    +                  s"not been fulfilled after $pendingAddTimeoutSeconds seconds")
    +                numExecutorsPendingToAdd = 0
    +                pendingAddExpirationTime = NOT_STARTED
    +              }
    +
    +              // Expire any outstanding pending remove requests that have timed out
    +              pendingRemoveExpirationTimes.foreach { case (executorId, expirationTime) =>
    +                if (now > expirationTime) {
    +                  logDebug(s"Expiring pending request to remove executor $executorId because " +
    +                    s"it has not been fulfilled after $pendingRemoveTimeoutSeconds seconds")
    +                  executorsPendingToRemove.remove(executorId)
    +                  pendingRemoveExpirationTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    --- End diff --
    
    This expression is quite complicated, so I'd break it up a bit:
    ```
    // Number to add if continuing exponential increase
    val targetNumExecutors = executorIds.size + numExecutorsPending + numExecutorsToAdd
    // Take into account max
    val adjustedTargetNumExecutors = math.min(targetNumExecutors, maxNumExecutors) 
    // Compute delta
    val adjustedNumExecutorsToAdd = adjustedTargetNumExecutors - numExistingExecutors
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18865383
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    --- End diff --
    
    Ok, I think that one sounds pretty reasonable too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18933408
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -110,19 +110,26 @@ private[spark] class TaskSetManager(
       // the one that it was launched from, but gets removed from them later.
       private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
    +  // List 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.
    +  // List of pending tasks for each rack -- similar to the above.
       private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set containing pending tasks with no locality preferences.
    +  // List of pending tasks with no locality preferences.
       var pendingTasksWithNoPrefs = new ArrayBuffer[Int]
     
    -  // Set containing all pending tasks (also used as a stack, as above).
    +  // List of all pending tasks (also used as a stack, as above).
       val allPendingTasks = new ArrayBuffer[Int]
     
    +  // Set of pending tasks used to keep track of whether more executors are needed
    --- End diff --
    
    My concern is that a stage may have many tasks, so removing from `allPendingTasks` will be fairly expensive, and we do this very often. We can't just make it a set because it's being used as a stack, so the ordering matters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60874816
  
      [Test build #22422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22422/consoleFull) for   PR 2746 at commit [`c60ec33`](https://github.com/apache/spark/commit/c60ec33219a9341fe5c3a7c25c148b37940ba1e3).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59308640
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21787/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59857732
  
    @kayousterhout @sryza Correct me if I'm wrong, but I believe grabbing as many executors as needed in MR/Tez comes for free in these two frameworks because a container is supposed to be short-lived (unless you explicitly specify to reuse containers). Thus, it is much easier to understand this policy in these contexts because the same policy is used for normal scheduling of tasks. In Spark, however, executors are mostly coarse-grained, so we have to use the ratio of pending tasks per executor to achieve something similar. However, this is more complicated for Spark because the correlation between number of executors added and the number of pending tasks these executors will be scheduled is less clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60475431
  
      [Test build #22207 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22207/consoleFull) for   PR 2746 at commit [`19c6c4b`](https://github.com/apache/spark/commit/19c6c4b7ad6b64f65475dc494a91c75226b72410).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61030936
  
      [Test build #22490 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22490/consoleFull) for   PR 2746 at commit [`8a4fdaa`](https://github.com/apache/spark/commit/8a4fdaad14c546f90acffef274355d884f7e6a59).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String)`
      * `  case class RequestExecutors(requestedTotal: Int) extends CoarseGrainedClusterMessage`
      * `  case class KillExecutors(executorIds: Seq[String]) extends CoarseGrainedClusterMessage`
      * `class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59863719
  
    > By "timer" I was referring to the logical timer, not java.util.Timer as an implementation detail.
    
    @andrewor14 , my bad, should have looked closer at the patch.
    
    @kayousterhout ,  @andrewor14 is exactly right.  MR/Tez containers are shorter lived, which gives elasticity at the expense of needing to suffer JVM startup times more often.  My point wasn't that "because they were able to pull it off, Spark should be able to pull it off" as much as "these frameworks have an ease-of-use advantage over Spark if we don't address this problem, and adding some complexity in Spark is worth closing that gap".  I think ease of use is more important than ease of understanding.  The main complaint I get with Spark is that it's difficult out of the box.
    
    With regards to ease of understanding, my opinion is that Kay's policy actually seems a little more straightforward.  I imagine that when a user needs to care about how this stuff works, the question they're probably asking: "as I watch this stage run and (go slow|take too many resources), is the number of executors I have reasonable?"  An answer based on the number of pending tasks is easier to grasp than an answer that changes through time and is based on exponentiation.  I.e. yes, the time/exponent rule is simple, but interpreting its behavior is less so.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60709621
  
      [Test build #22328 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22328/consoleFull) for   PR 2746 at commit [`92f36f9`](https://github.com/apache/spark/commit/92f36f99d2558e41eab72ed2e937185a45064bbc).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging `
      * `abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Logging `
      * `  class JobUIData(`
      * `public final class JavaStatusAPIDemo `
      * `  public static final class IdentityWithDelay<T> implements Function<T, T> `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19384212
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    --- End diff --
    
    Yeah I'll remove this once #2840 goes in and I rebase to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60552185
  
      [Test build #22272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22272/consoleFull) for   PR 2746 at commit [`149cc32`](https://github.com/apache/spark/commit/149cc32bee25b515da5a38edc2659af8aaaff2d2).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59856737
  
    > What's the reasoning behind using Timers vs. a polling approach? I think the latter is still a fair bit easier to understand if all else is equal.
    
    I think you're misunderstanding what I'm saying. We're still using the polling approach. By "timer" I was referring to the logical timer, not `java.util.Timer`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18932436
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logInfo(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    // Start the retry timer in case this addition fails
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total is $newTotalExecutors)")
    +    numExecutorsToAdd *= 2
    +    numExecutorsPendingToAdd += actualNumExecutorsToAdd
    +    backend.requestExecutors(actualNumExecutorsToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Retry a previous executor request that has not been fulfilled.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryAddExecutors(): Unit = synchronized {
    +    // Do not retry if there are no executors pending to be added (should never happen)
    +    if (numExecutorsPendingToAdd == 0) {
    +      logWarning("Attempted to retry adding executors when there are none pending to be added")
    +      cancelAddRetryTimer()
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    addRetryAttempts += 1
    +    if (addRetryAttempts > maxAddRetryAttempts) {
    +      logInfo(s"Giving up on adding $numExecutorsPendingToAdd executor(s) " +
    +        s"after $maxAddRetryAttempts failed attempts")
    +      numExecutorsPendingToAdd = 0
    +      // Also cancel original add timer because the cluster is not granting us new executors
    +      cancelAddTimer()
    +      return
    +    }
    +
    +    // Retry a previous request, then restart the retry timer in case this retry also fails
    +    logInfo(s"Previously requested executors have not all registered yet. " +
    +      s"Retrying to add $numExecutorsPendingToAdd executor(s) " +
    +      s"[attempt $addRetryAttempts/$maxAddRetryAttempts]")
    +    backend.requestExecutors(numExecutorsPendingToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Request the scheduler backend to decommission the given executor.
    +   * This expires the remove timer unless the executor is kept alive intentionally.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId) ||
    +        removeRetryAttempts.contains(executorId) ||
    +        retryRemoveTimers.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit")
    +      // Restart the remove timer to keep the executor marked as idle
    +      // Otherwise we won't be able to remove this executor even after new executors have joined
    +      startRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Send a kill request to the backend for this executor
    +    // Start the retry timer in case this removal fails
    +    logInfo(s"Removing executor $executorId because it has been idle for " +
    +      s"$removeThreshold seconds (new total is ${numExistingExecutors - 1})")
    +    executorsPendingToRemove.add(executorId)
    +    backend.killExecutor(executorId)
    +    cancelRemoveTimer(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Retry a previous attempt to decommission the given executor.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryRemoveExecutors(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to retry removing unknown executor $executorId")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if this executor is not pending to be killed (should never happen)
    +    if (!executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Attempted to retry removing executor $executorId when it's not to be removed!")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    removeRetryAttempts(executorId) =
    +      removeRetryAttempts.getOrElse(executorId, 0) + 1
    +    if (removeRetryAttempts(executorId) > maxRemoveRetryAttempts) {
    +      logInfo(s"Giving up on removing executor $executorId after " +
    +        s"$maxRemoveRetryAttempts failed attempts!")
    +      executorsPendingToRemove.remove(executorId)
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Retry a previous kill request for this executor
    +    // Restart the retry timer in case this retry also fails
    +    logInfo(s"Retrying previous attempt to remove $executorId " +
    +      s"[attempt ${removeRetryAttempts(executorId)}/$maxRemoveRetryAttempts]")
    +    backend.killExecutor(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPendingToAdd left)")
    +        if (numExecutorsPendingToAdd == 0) {
    +          logDebug("All previously pending executors have registered!")
    +          cancelAddRetryTimer()
    +        }
    +      }
    +      executorIds.add(executorId)
    +      startRemoveTimer(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      logDebug(s"Existing executor $executorId has been removed")
    +      executorIds.remove(executorId)
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +        cancelRemoveRetryTimer(executorId)
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Return whether the add timer is already running.
    +   */
    +  def isAddTimerRunning: Boolean = addTimerEnabled || addRetryTimerEnabled
    +
    +  /**
    +   * Return whether the remove timer for the given executor is already running.
    +   */
    +  def isRemoveTimerRunning(executorId: String): Boolean = {
    +    removeTimers.contains(executorId) || retryRemoveTimers.contains(executorId)
    +  }
    +
    +  /**
    +   * Start a timer to add executors, to expire in `addThreshold` seconds in the first
    +   * round, and `addInterval` seconds in every round thereafter. This is called when
    +   * the scheduler receives new pending tasks and the timer is not already started. This resets
    +   * the value of any existing add timer.
    +   */
    +  def startAddTimer(): Unit = {
    +    val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +    logDebug(s"Starting add executor timer (to expire in $threshold seconds)")
    +    addTimer = 0
    +    addTimerEnabled = true
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor, to expire in `removeThreshold` seconds.
    +   * This is called when an executor registers or finishes running tasks, and the timer is not
    +   * already started. This resets the value of any existing timer to remove this executor.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = {
    +    logDebug(s"Starting remove executor timer for $executorId " +
    +      s"(to expire in $removeThreshold seconds)")
    +    removeTimers(executorId) = 0
    --- End diff --
    
    No, if it's set to `removeThreshold` then you remove the executor right away. It's set to 0 to signal that the timer has just started, and this value is slowly incremented every loop iteration until it exceeds `removeThreshold` (and that's when you actually remove the executor).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19260365
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  def initialize(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +
    +              // Expire any outstanding pending add requests that have timed out
    +              if (pendingAddExpirationTime != NOT_STARTED && now >= pendingAddExpirationTime) {
    +                logDebug(s"Expiring all pending add requests because they have " +
    +                  s"not been fulfilled after $pendingAddTimeoutSeconds seconds")
    +                numExecutorsPendingToAdd = 0
    +                pendingAddExpirationTime = NOT_STARTED
    +              }
    +
    +              // Expire any outstanding pending remove requests that have timed out
    +              pendingRemoveExpirationTimes.foreach { case (executorId, expirationTime) =>
    +                if (now > expirationTime) {
    +                  logDebug(s"Expiring pending request to remove executor $executorId because " +
    +                    s"it has not been fulfilled after $pendingRemoveTimeoutSeconds seconds")
    +                  executorsPendingToRemove.remove(executorId)
    +                  pendingRemoveExpirationTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total will be $newTotalExecutors)")
    --- End diff --
    
    s/Adding/Requesting/g and  s/new total/new desired total/g to not to give false hope?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377740
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    --- End diff --
    
    can you declare the type here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60875890
  
      [Test build #22423 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22423/consoleFull) for   PR 2746 at commit [`a6a30f2`](https://github.com/apache/spark/commit/a6a30f2a0c5adcb5e9af253bf155870e0fa41f71).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59592148
  
    Hi all. I have discussed the design offline with @kayousterhout and @pwendell and we have come to the following high level consensus:
    
    - We should treat add as a best-effort thing. This means there is no need to retry it, and we shouldn't  wait for the new ones to register before asking for more. The latter point here means an exponential increase policy can become an add-to-max policy if we set the add interval to a small value.
    - The approach of determining the number of executors to add based on the number of pending tasks will be under consideration in the future, but will not be a part of this release. This is mainly because this add policy is more opaque to the user and the number added may be unpredictable depending on when the timer is triggered.
    - In a future release, we will make the scaling policies pluggable. Until then, for this release, we will expose a `@developerApi` `sc.addExecutors` and `sc.removeExecutors` in case the application wants to use this feature on their own (they won't have to enable `spark.dynamicAllocation.enabled` to use these).
    - We should assume that removes will always succeed for simplicity. This means there is no need to retry them.
    - To simplify the timer logic, we will make the variables hold the expiration time of the timer instead of a counter that is reset to 0 every time the timer triggers. This makes the semantics of the timer more easily understandable.
    - Use the listener API to identify when tasks are built up for testability.
    
    I should emphasize that this design is only for the first-cut implementation of this feature. We will make an effort to generalize this and expose the ability for the user to implement his/her own heuristics for 1.3 (tentative). Lastly, I will implement all of these shortly, and the new code will likely be quite different. Please kindly hold back your reviews until then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19451612
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    --- End diff --
    
    There is a JIRA about this https://issues.apache.org/jira/browse/SPARK-3859. It depends on when we want to start doing this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60983360
  
      [Test build #22461 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22461/consoleFull) for   PR 2746 at commit [`c79e907`](https://github.com/apache/spark/commit/c79e9079014776c28f041bfffdfaf4833dcae7e6).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922754
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    --- End diff --
    
    Yup. First N seconds (the threshold), then M seconds (the interval) every round thereafter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60314429
  
    @vanzin Yes, apparently so. I suppose it's fine to let `YarnAllocator` take care of this for us for now since we're only targeting Yarn at the moment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18920718
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    --- End diff --
    
    This is a nit but is it too annoying to just add the unit to the parameter name (e.g., spark.dynamicAllocation.addExecutorThresholdSeconds")? The extra time to type "seconds" seems significantly less than the time to lookup what the appropriate units are.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60343190
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22105/consoleFull) for   PR 2746 at commit [`092d1fd`](https://github.com/apache/spark/commit/092d1fdc9f8f450a447a3c8292a78eef9937ab3a).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61012643
  
      [Test build #22470 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22470/consoleFull) for   PR 2746 at commit [`c79e907`](https://github.com/apache/spark/commit/c79e9079014776c28f041bfffdfaf4833dcae7e6).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61030938
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22490/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60875897
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22423/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60862703
  
      [Test build #22412 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22412/consoleFull) for   PR 2746 at commit [`c3caa65`](https://github.com/apache/spark/commit/c3caa650108edde0b1386098368ad73e49bb3834).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18931227
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logInfo(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    // Start the retry timer in case this addition fails
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total is $newTotalExecutors)")
    +    numExecutorsToAdd *= 2
    +    numExecutorsPendingToAdd += actualNumExecutorsToAdd
    +    backend.requestExecutors(actualNumExecutorsToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Retry a previous executor request that has not been fulfilled.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryAddExecutors(): Unit = synchronized {
    +    // Do not retry if there are no executors pending to be added (should never happen)
    +    if (numExecutorsPendingToAdd == 0) {
    +      logWarning("Attempted to retry adding executors when there are none pending to be added")
    +      cancelAddRetryTimer()
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    addRetryAttempts += 1
    +    if (addRetryAttempts > maxAddRetryAttempts) {
    +      logInfo(s"Giving up on adding $numExecutorsPendingToAdd executor(s) " +
    +        s"after $maxAddRetryAttempts failed attempts")
    +      numExecutorsPendingToAdd = 0
    +      // Also cancel original add timer because the cluster is not granting us new executors
    +      cancelAddTimer()
    +      return
    +    }
    +
    +    // Retry a previous request, then restart the retry timer in case this retry also fails
    +    logInfo(s"Previously requested executors have not all registered yet. " +
    +      s"Retrying to add $numExecutorsPendingToAdd executor(s) " +
    +      s"[attempt $addRetryAttempts/$maxAddRetryAttempts]")
    +    backend.requestExecutors(numExecutorsPendingToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Request the scheduler backend to decommission the given executor.
    +   * This expires the remove timer unless the executor is kept alive intentionally.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId) ||
    +        removeRetryAttempts.contains(executorId) ||
    +        retryRemoveTimers.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit")
    +      // Restart the remove timer to keep the executor marked as idle
    +      // Otherwise we won't be able to remove this executor even after new executors have joined
    +      startRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Send a kill request to the backend for this executor
    +    // Start the retry timer in case this removal fails
    +    logInfo(s"Removing executor $executorId because it has been idle for " +
    +      s"$removeThreshold seconds (new total is ${numExistingExecutors - 1})")
    +    executorsPendingToRemove.add(executorId)
    +    backend.killExecutor(executorId)
    +    cancelRemoveTimer(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Retry a previous attempt to decommission the given executor.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryRemoveExecutors(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to retry removing unknown executor $executorId")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if this executor is not pending to be killed (should never happen)
    +    if (!executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Attempted to retry removing executor $executorId when it's not to be removed!")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    removeRetryAttempts(executorId) =
    +      removeRetryAttempts.getOrElse(executorId, 0) + 1
    +    if (removeRetryAttempts(executorId) > maxRemoveRetryAttempts) {
    +      logInfo(s"Giving up on removing executor $executorId after " +
    +        s"$maxRemoveRetryAttempts failed attempts!")
    +      executorsPendingToRemove.remove(executorId)
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Retry a previous kill request for this executor
    +    // Restart the retry timer in case this retry also fails
    +    logInfo(s"Retrying previous attempt to remove $executorId " +
    +      s"[attempt ${removeRetryAttempts(executorId)}/$maxRemoveRetryAttempts]")
    +    backend.killExecutor(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPendingToAdd left)")
    +        if (numExecutorsPendingToAdd == 0) {
    +          logDebug("All previously pending executors have registered!")
    +          cancelAddRetryTimer()
    +        }
    +      }
    +      executorIds.add(executorId)
    +      startRemoveTimer(executorId)
    --- End diff --
    
    Hmmm... maybe this is explained somewhere else in the code, but why start a remove timer right when the executor is registered? Shouldn't you wait for some of the events described elsewhere to happen for that (like all tasks on that executor finishing)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59136518
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21745/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60705774
  
      [Test build #22328 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22328/consoleFull) for   PR 2746 at commit [`92f36f9`](https://github.com/apache/spark/commit/92f36f99d2558e41eab72ed2e937185a45064bbc).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61013967
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22478/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59136744
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21750/consoleFull) for   PR 2746 at commit [`baaa403`](https://github.com/apache/spark/commit/baaa403b16d10107abeeb1fba721b2c301d2c0aa).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18732711
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors
    +  private val addExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60) // s
    +  private val addExecutorInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addExecutorThreshold) // s
    +  private val removeExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 300) // s
    +
    +  // Timers that keep track of when to add and remove executors
    +  private var addExecutorTimer: Option[Timer] = None
    +  private val removeExecutorTimers: mutable.Map[String, Timer] = new mutable.HashMap[String, Timer]
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // The number of pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private var numExecutorsPendingToRemove = 0
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String] ++= scheduler.executorIdToHost.keys
    +
    +  // Start idle timer for all new executors
    +  synchronized { executorIds.foreach(startRemoveExecutorTimer) }
    +
    +  /**
    +   * Start the add executor timer if it does not already exist.
    +   * This is called when a new pending task is added. The add is then triggered
    +   * if the pending tasks queue is not drained in `addExecutorThreshold` seconds.
    +   */
    +  def startAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorThreshold)
    +
    +  /**
    +   * Restart the add executor timer.
    +   * This is called when the previous add executor timer has expired but not canceled. The add
    +   * is then triggered again if all pending executors from the previous round have registered,
    +   * and the pending tasks queue is still not drained in `addExecutorInterval` seconds.
    +   */
    +  private def restartAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorInterval)
    +
    +  /**
    +   * Start the add executor timer using the given delay if the timer does not already exist.
    +   */
    +  private def startAddExecutorTimer(timerDelaySeconds: Long): Unit = {
    --- End diff --
    
    I don't see it documented in the style guidelines, but the convention I've observed for methods with no return types is to omit the ": Unit =".  Any reason for including it in the methods here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59869868
  
    I think the main objection @pwendell and I have against the policy is not that it is complicated in terms of implementation, but that it is complicated in terms of its resulting behavior. The main reason for this complexity, then, is because the policy does not take into account the running time of each task.
    
    In Spark, the most common model of execution is that a stage has many short-running tasks. Under this policy, we will try to add as many executors as possible to run all remaining pending tasks in parallel. However, once we do this, each task likely finishes promptly and we will have many executors that are now idle. In other words, under this kind of workload, this policy is basically equivalent to the "add-to-max" policy.
    
    Even for longer-running tasks, we still have a problem. If the add timer expires multiple times while these tasks are being run, we may end up double counting the number of executors needed. This is because each round of addition thinks the remaining pending tasks have not already been accounted for. In this case, the semantics of how many executors are added relative to how many pending tasks are outstanding are less clear to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60196383
  
    @sryza just so I understand. I tell YARN I want 10 executors to be pending. Then say YARN grants me two executors. Does it internally decrement the pending number to 8 (and can I read back that state?). Or could we just infer that it has decremented the counter based on getting new executors? How would it work?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19194532
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    --- End diff --
    
    can you declare the type here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59853416
  
    Everything sounds good except for a couple specific callouts below:
    
    > we shouldn't wait for the new ones to register before asking for more.
    This is still worrying to me.  My concern is that Spark will try to grab way more executors than it needs if YARN suddenly makes resources available to it in the middle of a long stage.  Certainly a user can avoid this by setting a max, but places an additional configuration burden that I think is both avoidable and difficult to reason about for inexperienced users.  Sorry to continually be so picky about this - my main concern about this stuff is that similar frameworks like MR and Tez essentially provide full resource elasticity with zero attention to configuration required on behalf of the user.  Their elasticity of course comes at the price of incurring more JVM startups, but I think additional complexity is worthwhile if it can get Spark to achieve parity.
    
    > To simplify the timer logic, we will make the variables hold the expiration times of the timers instead of counters that are reset to 0 every time the timers trigger.
    What's the reasoning behind using Timers vs. a polling approach?  I think the latter is still a fair bit easier to understand if all else is equal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19195258
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    --- End diff --
    
    what about just `numExecutorsPending`? "To add" seems imperative to me, i.e. I should add these later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922212
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -101,7 +101,7 @@ private[spark] class TaskSetManager(
       // TODO: We should kill any running task attempts when the task set manager becomes a zombie.
       var isZombie = false
     
    -  // Set of pending tasks for each executor. These collections are actually
    +  // List of pending tasks for each executor. These collections are actually
    --- End diff --
    
    :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61012655
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22470/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59295974
  
    I agree with @pwendell that the retry thing looks a little convoluted. Especially since I don't see any way for the backend to report back the status of requests (e.g. has this executor request failed or is it just taking a long time?).
    
    It would be nice to have this channel back so that a proper status can be reported by the backend. e.g. let's say that you hit some limit in the number of executors you can allocate, the current code will keep retrying that many times fruitlessly until it decides to give up. That could be avoided if the backend could just say "I can't allocate more executors".
    
    Also, it doesn't seem like there's a way for the backend to tell whether a `requestExecutor` call is a new one or a retry. For Yarn, at least, that makes a difference; you don't want to create new requests if you're retrying, or you'll end up allocating more executors than intended. (This kinda loops back into the comment above about communicating request status back to the requestor.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60119850
  
    @sryza - ah okay awesome. So think that case we can just "update" YARN with the new number of executors we'd like to be pending.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60120863
  
    In that case it might make sense for the interface to the schedulers to be `setRequestedExecutors(num)` rather than `addExecutors(num)`. This makes more sense for both YARN and standalone and it will make things simpler. We can always just compute the number we'd like to have (based on the current number we have, the maximum, etc).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19509810
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if the time then has exceeded any of the
    +   * add and remove times that are set. If so, it triggers the corresponding action.
    +   */
    +  private def startPolling(): Unit = {
    --- End diff --
    
    Also, I'm not so sure about doing this through a thread pool. The existing code keeps the number of threads used by the allocation manager to a minimum to avoid contention. I can factor out the clock for testing, but I don't see a nicer way to rewrite the logic into a separate function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377690
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    --- End diff --
    
    For this comment - not sure it makes sense to put it in the code (rather than in the PR description).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18931177
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    --- End diff --
    
    Could you clarify in the comment what does the timer value mean? Is it the absolute time when the timer should trigger, or a timeout (relative to what?)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19387388
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd *= 2
    +      numExecutorsPending += actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are " +
    +        s"only $minNumExecutors executor(s) left, which is the limit")
    +      return
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be triggered
    +   * in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in every round
    +   * thereafter. This is called when the scheduler receives new pending tasks.
    +   */
    +  def startAddTimer(): Unit = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    +    logDebug(s"Canceling remove executor timer for $executorId")
    +    removeTimes.remove(executorId)
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  private val NOT_STARTED = -1L
    +}
    +
    +/**
    + * A listener that notifies the given allocation manager of when to add and remove executors.
    + */
    +private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +  extends SparkListener {
    +
    +  private val stageIdToPendingTaskIndex = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +  private val executorIdToTaskId = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
    +    val stageId = stageSubmitted.stageInfo.stageId
    +    val numTasks = stageSubmitted.stageInfo.numTasks
    +    // Start the add timer because there are new pending tasks
    +    stageIdToPendingTaskIndex.getOrElseUpdate(
    +      stageId, new mutable.HashSet[Int]) ++= (0 to numTasks - 1)
    --- End diff --
    
    Good point...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19558120
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -0,0 +1,661 @@
    +/*
    + * 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
    +
    +import org.scalatest.{FunSuite, PrivateMethodTester}
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.scheduler._
    +import org.apache.spark.storage.BlockManagerId
    +
    +/**
    + * Test add and remove behavior of ExecutorAllocationManager.
    + */
    +class ExecutorAllocationManagerSuite extends FunSuite {
    +  import ExecutorAllocationManager._
    +  import ExecutorAllocationManagerSuite._
    +
    +  test("verify min/max executors") {
    +    // No min or max
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +    intercept[SparkException] { new SparkContext(conf) }
    +
    +    // Only min
    +    val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
    +    intercept[SparkException] { new SparkContext(conf1) }
    +
    +    // Only max
    +    val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
    +    intercept[SparkException] { new SparkContext(conf2) }
    +
    +    // Both min and max, but min > max
    +    intercept[SparkException] { createSparkContext(2, 1) }
    +
    +    // Both min and max, and min == max
    +    val sc1 = createSparkContext(1, 1)
    +    assert(sc1.executorAllocationManager.isDefined)
    +    sc1.stop()
    +
    +    // Both min and max, and min < max
    +    val sc2 = createSparkContext(1, 2)
    +    assert(sc2.executorAllocationManager.isDefined)
    +    sc2.stop()
    +  }
    +
    +  test("starting state") {
    +    val sc = createSparkContext()
    +    val manager = sc.executorAllocationManager.get
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(executorIds(manager).isEmpty)
    +    assert(addTime(manager) === ExecutorAllocationManager.NOT_SET)
    +    assert(removeTimes(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test("add executors") {
    +    val sc = createSparkContext(1, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Keep adding until the limit is reached
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 1)
    +    assert(numExecutorsPending(manager) === 1)
    +    assert(numExecutorsToAdd(manager) === 2)
    +    assert(addExecutors(manager) === 2)
    +    assert(numExecutorsPending(manager) === 3)
    +    assert(numExecutorsToAdd(manager) === 4)
    +    assert(addExecutors(manager) === 4)
    +    assert(numExecutorsPending(manager) === 7)
    +    assert(numExecutorsToAdd(manager) === 8)
    +    assert(addExecutors(manager) === 3) // reached the limit of 10
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 10)
    +    assert(numExecutorsToAdd(manager) === 1)
    +
    +    // Register previously requested executors
    +    onExecutorAdded(manager, "first")
    +    assert(numExecutorsPending(manager) === 9)
    +    onExecutorAdded(manager, "second")
    +    onExecutorAdded(manager, "third")
    +    onExecutorAdded(manager, "fourth")
    +    assert(numExecutorsPending(manager) === 6)
    +    onExecutorAdded(manager, "first") // duplicates should not count
    +    onExecutorAdded(manager, "second")
    +    assert(numExecutorsPending(manager) === 6)
    +
    +    // Try adding again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    assert(addExecutors(manager) === 0)
    +    assert(numExecutorsPending(manager) === 6)
    +    assert(numExecutorsToAdd(manager) === 1)
    +    sc.stop()
    +  }
    +
    +  test("remove executors") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +    (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) }
    +
    +    // Keep removing until the limit is reached
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(removeExecutor(manager, "1"))
    +    assert(executorsPendingToRemove(manager).size === 1)
    +    assert(executorsPendingToRemove(manager).contains("1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(removeExecutor(manager, "3"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).contains("2"))
    +    assert(executorsPendingToRemove(manager).contains("3"))
    +    assert(!removeExecutor(manager, "100")) // remove non-existent executors
    +    assert(!removeExecutor(manager, "101"))
    +    assert(executorsPendingToRemove(manager).size === 3)
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(!removeExecutor(manager, "6")) // reached the limit of 5
    +    assert(executorsPendingToRemove(manager).size === 5)
    +    assert(executorsPendingToRemove(manager).contains("4"))
    +    assert(executorsPendingToRemove(manager).contains("5"))
    +    assert(!executorsPendingToRemove(manager).contains("6"))
    +
    +    // Kill executors previously requested to remove
    +    onExecutorRemoved(manager, "1")
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("1"))
    +    onExecutorRemoved(manager, "2")
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    assert(!executorsPendingToRemove(manager).contains("2"))
    +    assert(!executorsPendingToRemove(manager).contains("3"))
    +    onExecutorRemoved(manager, "2") // duplicates should not count
    +    onExecutorRemoved(manager, "3")
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    onExecutorRemoved(manager, "4")
    +    onExecutorRemoved(manager, "5")
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +
    +    // Try removing again
    +    // This should still fail because the number pending + running is still at the limit
    +    assert(!removeExecutor(manager, "7"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    assert(!removeExecutor(manager, "8"))
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    sc.stop()
    +  }
    +
    +  test ("interleaving add and remove") {
    +    val sc = createSparkContext(5, 10)
    +    val manager = sc.executorAllocationManager.get
    +
    +    // Add a few executors
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 2)
    +    assert(addExecutors(manager) === 4)
    +    onExecutorAdded(manager, "1")
    +    onExecutorAdded(manager, "2")
    +    onExecutorAdded(manager, "3")
    +    onExecutorAdded(manager, "4")
    +    onExecutorAdded(manager, "5")
    +    onExecutorAdded(manager, "6")
    +    onExecutorAdded(manager, "7")
    +    assert(executorIds(manager).size === 7)
    +
    +    // Remove until limit
    +    assert(removeExecutor(manager, "1"))
    +    assert(removeExecutor(manager, "2"))
    +    assert(!removeExecutor(manager, "3")) // lower limit reached
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorRemoved(manager, "1")
    +    onExecutorRemoved(manager, "2")
    +    assert(executorIds(manager).size === 5)
    +
    +    // Add until limit
    +    assert(addExecutors(manager) === 5) // upper limit reached
    +    assert(addExecutors(manager) === 0)
    +    assert(!removeExecutor(manager, "3")) // still at lower limit
    +    assert(!removeExecutor(manager, "4"))
    +    onExecutorAdded(manager, "8")
    +    onExecutorAdded(manager, "9")
    +    onExecutorAdded(manager, "10")
    +    onExecutorAdded(manager, "11")
    +    onExecutorAdded(manager, "12")
    +    assert(executorIds(manager).size === 10)
    +
    +    // Remove succeeds again, now that we are no longer at the lower limit
    +    assert(removeExecutor(manager, "3"))
    +    assert(removeExecutor(manager, "4"))
    +    assert(removeExecutor(manager, "5"))
    +    assert(removeExecutor(manager, "6"))
    +    assert(executorIds(manager).size === 10)
    +    assert(addExecutors(manager) === 0) // still at upper limit
    +    onExecutorRemoved(manager, "3")
    +    onExecutorRemoved(manager, "4")
    +    assert(executorIds(manager).size === 8)
    +
    +    // Add succeeds again, now that we are no longer at the upper limit
    +    // Number of executors added restarts at 1
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    assert(executorIds(manager).size === 8)
    +    onExecutorRemoved(manager, "5")
    +    onExecutorRemoved(manager, "6")
    +    onExecutorAdded(manager, "13")
    +    onExecutorAdded(manager, "14")
    +    assert(executorIds(manager).size === 8)
    +    assert(addExecutors(manager) === 1)
    +    assert(addExecutors(manager) === 1) // upper limit reached again
    +    assert(addExecutors(manager) === 0)
    +    onExecutorAdded(manager, "15")
    +    onExecutorAdded(manager, "16")
    +    assert(executorIds(manager).size === 10)
    +    sc.stop()
    +  }
    +
    +  test("starting/canceling add timer") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(8888L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting add timer is idempotent
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val firstAddTime = addTime(manager)
    +    assert(firstAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime) // timer is already started
    +    clock.tick(200L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === firstAddTime)
    +    onSchedulerQueueEmpty(manager)
    +
    +    // Restart add timer
    +    clock.tick(1000L)
    +    assert(addTime(manager) === NOT_SET)
    +    onSchedulerBacklogged(manager)
    +    val secondAddTime = addTime(manager)
    +    assert(secondAddTime === clock.getTime + schedulerBacklogTimeout * 1000)
    +    clock.tick(100L)
    +    onSchedulerBacklogged(manager)
    +    assert(addTime(manager) === secondAddTime) // timer is already started
    +    assert(addTime(manager) !== firstAddTime)
    +    assert(firstAddTime !== secondAddTime)
    +  }
    +
    +  test("starting/canceling remove timers") {
    +    val sc = createSparkContext(2, 10)
    +    val clock = new TestClock(14444L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Starting remove timer is idempotent for each executor
    +    assert(removeTimes(manager).isEmpty)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("1"))
    +    val firstRemoveTime = removeTimes(manager)("1")
    +    assert(firstRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(100L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started
    +    clock.tick(200L)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager)("1") === firstRemoveTime)
    +    clock.tick(300L)
    +    onExecutorIdle(manager, "2")
    +    assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor
    +    assert(removeTimes(manager)("2") === clock.getTime + executorIdleTimeout * 1000)
    +    clock.tick(400L)
    +    onExecutorIdle(manager, "3")
    +    assert(removeTimes(manager)("3") !== firstRemoveTime)
    +    assert(removeTimes(manager)("3") === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("2"))
    +    assert(removeTimes(manager).contains("3"))
    +
    +    // Restart remove timer
    +    clock.tick(1000L)
    +    onExecutorBusy(manager, "1")
    +    assert(removeTimes(manager).size === 2)
    +    onExecutorIdle(manager, "1")
    +    assert(removeTimes(manager).size === 3)
    +    assert(removeTimes(manager).contains("1"))
    +    val secondRemoveTime = removeTimes(manager)("1")
    +    assert(secondRemoveTime === clock.getTime + executorIdleTimeout * 1000)
    +    assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started
    +    assert(removeTimes(manager)("1") !== firstRemoveTime)
    +    assert(firstRemoveTime !== secondRemoveTime)
    +  }
    +
    +  test("mock polling loop with no events") {
    +    val sc = createSparkContext(1, 20)
    +    val manager = sc.executorAllocationManager.get
    +    val clock = new TestClock(2020L)
    +    manager.setClock(clock)
    +
    +    // No events - we should not be adding or removing
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(100L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(1000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(10000L)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +  }
    +
    +  test("mock polling loop add behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Scheduler queue backlogged
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 0) // timer not exceeded yet
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // first timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded
    +
    +    // Scheduler queue drained
    +    onSchedulerQueueEmpty(manager)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7) // timer is canceled
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7)
    +
    +    // Scheduler queue backlogged again
    +    onSchedulerBacklogged(manager)
    +    clock.tick(schedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1) // timer restarted
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4)
    +    clock.tick(sustainedSchedulerBacklogTimeout * 1000)
    +    schedule(manager)
    +    assert(numExecutorsPending(manager) === 20) // limit reached
    +  }
    +
    +  test("mock polling loop remove behavior") {
    +    val sc = createSparkContext(1, 20)
    +    val clock = new TestClock(2020L)
    +    val manager = sc.executorAllocationManager.get
    +    manager.setClock(clock)
    +
    +    // Remove idle executors on timeout
    +    onExecutorAdded(manager, "executor-1")
    +    onExecutorAdded(manager, "executor-2")
    +    onExecutorAdded(manager, "executor-3")
    +    assert(removeTimes(manager).size === 3)
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000 / 2)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // idle threshold not reached yet
    +    assert(executorsPendingToRemove(manager).isEmpty)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle threshold exceeded
    +    assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining)
    +
    +    // Mark a subset as busy - only idle executors should be removed
    +    onExecutorAdded(manager, "executor-4")
    +    onExecutorAdded(manager, "executor-5")
    +    onExecutorAdded(manager, "executor-6")
    +    onExecutorAdded(manager, "executor-7")
    +    assert(removeTimes(manager).size === 5)              // 5 active executors
    +    assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed
    +    onExecutorBusy(manager, "executor-4")
    +    onExecutorBusy(manager, "executor-5")
    +    onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones)
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 2) // remove only idle executors
    +    assert(!removeTimes(manager).contains("executor-4"))
    +    assert(!removeTimes(manager).contains("executor-5"))
    +    assert(!removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 2)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty) // idle executors are removed
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    assert(!executorsPendingToRemove(manager).contains("executor-4"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-5"))
    +    assert(!executorsPendingToRemove(manager).contains("executor-6"))
    +
    +    // Busy executors are now idle and should be removed
    +    onExecutorIdle(manager, "executor-4")
    +    onExecutorIdle(manager, "executor-5")
    +    onExecutorIdle(manager, "executor-6")
    +    schedule(manager)
    +    assert(removeTimes(manager).size === 3) // 0 busy and 3 idle
    +    assert(removeTimes(manager).contains("executor-4"))
    +    assert(removeTimes(manager).contains("executor-5"))
    +    assert(removeTimes(manager).contains("executor-6"))
    +    assert(executorsPendingToRemove(manager).size === 4)
    +    clock.tick(executorIdleTimeout * 1000)
    +    schedule(manager)
    +    assert(removeTimes(manager).isEmpty)
    +    assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining)
    +  }
    +
    +  test("listeners trigger add executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Starting a stage should start the add timer
    +    val numTasks = 10
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting a subset of the tasks should not cancel the add timer
    +    val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") }
    +    taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +
    +    // Starting all remaining tasks should cancel the add timer
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head))
    +    assert(addTime(manager) === NOT_SET)
    +
    +    // Start two different stages
    +    // The add timer should be canceled only if all tasks in both stages start running
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks)))
    +    sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks)))
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) }
    +    assert(addTime(manager) !== NOT_SET)
    +    taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) }
    +    assert(addTime(manager) === NOT_SET)
    +  }
    +
    +  test("listeners trigger remove executors correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // Added executors should start the remove timers for each executor
    +    (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) }
    +    assert(removeTimes(manager).size === 5)
    +
    +    // Starting a task cancel the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1")))
    +    sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2")))
    +    assert(removeTimes(manager).size === 3)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +    assert(!removeTimes(manager).contains("executor-2"))
    +
    +    // Finishing all tasks running on an executor should start the remove timer for that executor
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 4)
    +    assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet
    +    assert(removeTimes(manager).contains("executor-2"))
    +    sc.listenerBus.postToAll(SparkListenerTaskEnd(
    +      0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics))
    +    assert(removeTimes(manager).size === 5)
    +    assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished
    +  }
    +
    +  test("listeners trigger add and remove executor callbacks correctly") {
    +    val sc = createSparkContext(2, 10)
    +    val manager = sc.executorAllocationManager.get
    +    assert(executorIds(manager).isEmpty)
    +    assert(removeTimes(manager).isEmpty)
    +
    +    // New executors have registered
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-1", "host1", 1), 100L))
    +    assert(executorIds(manager).size === 1)
    +    assert(executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(removeTimes(manager).contains("executor-1"))
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerAdded(
    +      0L, BlockManagerId("executor-2", "host2", 1), 100L))
    +    assert(executorIds(manager).size === 2)
    +    assert(executorIds(manager).contains("executor-2"))
    +    assert(removeTimes(manager).size === 2)
    +    assert(removeTimes(manager).contains("executor-2"))
    +
    +    // Existing executors have disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-1", "host1", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(!executorIds(manager).contains("executor-1"))
    +    assert(removeTimes(manager).size === 1)
    +    assert(!removeTimes(manager).contains("executor-1"))
    +
    +    // Unknown executor has disconnected
    +    sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved(
    +      0L, BlockManagerId("executor-3", "host3", 1)))
    +    assert(executorIds(manager).size === 1)
    +    assert(removeTimes(manager).size === 1)
    +  }
    +
    +}
    +
    +/**
    + * Helper methods for testing ExecutorAllocationManager.
    + * This includes methods to access private methods and fields in ExecutorAllocationManager.
    + */
    +private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
    +  private val schedulerBacklogTimeout = 1L
    +  private val sustainedSchedulerBacklogTimeout = 2L
    +  private val executorIdleTimeout = 3L
    +
    +  private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = {
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test-executor-allocation-manager")
    +      .set("spark.dynamicAllocation.enabled", "true")
    +      .set("spark.dynamicAllocation.minExecutors", minExecutors + "")
    +      .set("spark.dynamicAllocation.maxExecutors", maxExecutors + "")
    +      .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout",
    +        sustainedSchedulerBacklogTimeout + "")
    +      .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout + "")
    +    new SparkContext(conf)
    +  }
    +
    +  private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = {
    +    new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details")
    +  }
    +
    +  private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = {
    +    new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false)
    +  }
    +
    +  /* ------------------------------------------------------- *
    +   | Helper methods for accessing private methods and fields |
    --- End diff --
    
    Man, this is a lot of extra code just to be able to keep methods private... sorry you had to go through that.
    
    In Java-land here we use `com.google.common.annotations.VisibleForTesting` and just change the method/field's visibility. The annotation serves as a guide ("don't call this if you're not writing a test").


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18739007
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    --- End diff --
    
    Though I think we need some notion of `executor` in there. `DynamicExecutorAllocationManager`? `ExecutorAllocationManager`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58737471
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21616/consoleFull) for   PR 2746 at commit [`b3c7d44`](https://github.com/apache/spark/commit/b3c7d446160747b79e6afbd844f9c8b6d0158781).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18908858
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    --- End diff --
    
    A lot of the logic in here seems related to retries. Why do we have to deal with retrying at all? What if we just do individual executor requests one at a time and we don't do any dynamic scaling until we get back the result of the last request? Could this simplify a lot of complexity here? My understanding is that YARN etc should be able to launch executors in order seconds, so we could add ten or more executors per minute... that seems pretty reasonable for the type of workloads this is targeting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18921725
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    --- End diff --
    
    Would it be better to calculate the actual sleep time using the clock, preferrably a monotonic one (like `System.nanoTime`)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59138007
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21747/consoleFull) for   PR 2746 at commit [`9bcc0bc`](https://github.com/apache/spark/commit/9bcc0bca1156ce086462ceba162c93310224ed1e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18921891
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    --- End diff --
    
    Not sure why this is being reset here. IIUC you're just delaying the next round until the previous round has been completely added; but you still want to follow the exponential increase when you actually start the next round, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18920962
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    --- End diff --
    
    <3 units in the name here, but can you do "Millis" at least?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59562701
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21859/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19195273
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    --- End diff --
    
    Would it be better to store the number added in the previous round rather than the number that _should_ be added in the next round. It's a bit confusing because we won't actually add this number in some cases. (on this one I didn't think through all the implications, so it might be simpler as-is).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19426487
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    --- End diff --
    
    No I plan to do that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60036315
  
    The latest changes reflect most of the design I have outlined in my bullet point post above. I intend to refactor it a little more by accessing the task information through the listener interface.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18932455
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    --- End diff --
    
    Yeah I'll add a comment. Looks like multiple people here find this confusing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19327200
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    --- End diff --
    
    Any particular rationale for these defaults?  My intuition would be that the addThreshold should be something close to how long it takes for a new executor to launch and drain its capacity in tasks from the queue.  This shouldn't take more than a couple seconds.
    
    A remove threshold of 10 minutes also seems high to me. The threshold should be something like a standard deviation up from the typical amount of time that a user would wait before running another query / job after one completes.  I'd imagine a minute or so would suffice. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59854796
  
    One approach worth considering that I think would require less user involvement would be to set a limit on the number of outstanding requests instead of on the total number of executors.  The limit could be specified as a fraction of the current allocation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19321383
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    --- End diff --
    
    Yeah I'll do that when I merge this with #2840 (whichever goes in later)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19427380
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    --- End diff --
    
    I understand that. In fact `YarnAllocator` already has a thread that periodically heartbeats requests to the RM. The acknowledgment here refers to the `AM` acknowledging that it has received this request from the driver, but will asynchronously forward the request to the `RM` later. The assumption here is that once the `AM` receives the request, it is guaranteed to eventually forward it to the `RM` after translating it according to the AM-RM interface (through some retry logic already handled in `YarnAllocator`). In other words, in Yarn mode the driver treats the `AM/RM` pair as the "cluster manager", and all it wants is an acknowledgment that the request has been received (but not necessarily fulfilled).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19389583
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   * Return whether the executor is actually requested to be removed. Exposed for testing.
    +   */
    +  def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left, which is less than the limit $minNumExecutors")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added. Exposed for testing.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed. Exposed for testing.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be
    +   * triggered in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in
    +   * every round thereafter. This is called when the scheduler receives new pending tasks.
    +   * Return the value of the add timer. Exposed for testing.
    +   */
    +  def startAddTimer(): Long = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +    addTime
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   * Return the value of the remove timer. Exposed for testing.
    +   */
    +  def startRemoveTimer(executorId: String): Long = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +    removeTimes(executorId)
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left. Exposed for testing.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task. Exposed for testing.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    +    logDebug(s"Canceling remove executor timer for $executorId")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /* --------------------------- *
    +   | Getters exposed for testing |
    +   * --------------------------- */
    +
    +  def getNumExecutorsToAdd: Int = numExecutorsToAdd
    +  def getNumExecutorsPending: Int = numExecutorsPending
    +  def getExecutorsPendingToRemove: collection.Set[String] = executorsPendingToRemove
    +  def getExecutorIds: collection.Set[String] = executorIds
    +  def getAddTime: Long = addTime
    +  def getRemoveTimes: collection.Map[String, Long] = removeTimes
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  private[spark] val NOT_STARTED = -1L
    --- End diff --
    
    If the object ExecutorAllocationManager is already private, it it necessary to declare NOT_STARTED as so?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18732461
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    --- End diff --
    
    Consider calling this DynamicAllocationManager?  "Executor scaling" sound to me like making executors more scalable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18733008
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors
    +  private val addExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60) // s
    +  private val addExecutorInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addExecutorThreshold) // s
    +  private val removeExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 300) // s
    +
    +  // Timers that keep track of when to add and remove executors
    +  private var addExecutorTimer: Option[Timer] = None
    +  private val removeExecutorTimers: mutable.Map[String, Timer] = new mutable.HashMap[String, Timer]
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // The number of pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private var numExecutorsPendingToRemove = 0
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String] ++= scheduler.executorIdToHost.keys
    +
    +  // Start idle timer for all new executors
    +  synchronized { executorIds.foreach(startRemoveExecutorTimer) }
    +
    +  /**
    +   * Start the add executor timer if it does not already exist.
    +   * This is called when a new pending task is added. The add is then triggered
    +   * if the pending tasks queue is not drained in `addExecutorThreshold` seconds.
    +   */
    +  def startAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorThreshold)
    +
    +  /**
    +   * Restart the add executor timer.
    +   * This is called when the previous add executor timer has expired but not canceled. The add
    +   * is then triggered again if all pending executors from the previous round have registered,
    +   * and the pending tasks queue is still not drained in `addExecutorInterval` seconds.
    +   */
    +  private def restartAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorInterval)
    +
    +  /**
    +   * Start the add executor timer using the given delay if the timer does not already exist.
    +   */
    +  private def startAddExecutorTimer(timerDelaySeconds: Long): Unit = {
    +    addExecutorTimer.synchronized {
    +      if (addExecutorTimer.isEmpty) {
    +        logDebug(s"Starting add executor timer (to expire in $timerDelaySeconds seconds)")
    +        addExecutorTimer = Some(new Timer)
    +        addExecutorTimer.get.schedule(
    +          new AddExecutorTimerTask(numExecutorsToAdd), timerDelaySeconds * 1000)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor if the timer does not already exist.
    +   * This is called when the executor initially registers with the driver or finishes running
    +   * a task. The removal is then triggered if the executor stays idle (i.e. not running a task)
    +   * for `removeExecutorThreshold` seconds.
    +   */
    +  def startRemoveExecutorTimer(executorId: String): Unit = {
    +    removeExecutorTimers.synchronized {
    +      if (!removeExecutorTimers.contains(executorId)) {
    +        logDebug(s"Starting idle timer for executor $executorId " +
    +          s"(to expire in $removeExecutorThreshold seconds)")
    +        removeExecutorTimers(executorId) = new Timer
    +        removeExecutorTimers(executorId).schedule(
    +          new RemoveExecutorTimerTask(executorId), removeExecutorThreshold * 1000)
    +      }
    +    }
    +    // Acquire a more general lock here because we might mutate `executorId`
    +    synchronized {
    +      if (!executorIds.contains(executorId)) {
    +        logWarning(s"Started idle timer for unknown executor $executorId.")
    +        executorIds.add(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that adds executors.
    +   * This is called when the pending task queue is drained.
    +   */
    +  def cancelAddExecutorTimer(): Unit = addExecutorTimer.synchronized {
    +    addExecutorTimer.foreach { timer =>
    +      logDebug("Canceling add executor timer because task queue is drained!")
    +      timer.cancel()
    +      numExecutorsToAdd = 1
    +      addExecutorTimer = None
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing timer that removes the given executor.
    +   * This is called when the executor is no longer idle.
    +   */
    +  def cancelRemoveExecutorTimer(executorId: String): Unit = removeExecutorTimers.synchronized {
    +    if (removeExecutorTimers.contains(executorId)) {
    +      logDebug(s"Canceling idle timer for executor $executorId.")
    +      removeExecutorTimers(executorId).cancel()
    +      removeExecutorTimers.remove(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Negotiate with the scheduler backend to add new executors.
    +   * This ensures the resulting number of executors is correctly constrained by the upper bound.
    +   * Return the number of executors actually requested.
    +   */
    +  private def addExecutors(numExecutorsRequested: Int): Int = synchronized {
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    val numExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsRequested <= maxNumExecutors) {
    +        numExecutorsRequested
    +      } else {
    +        // Add just enough to reach `maxNumExecutors`
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newNumExecutors = numExistingExecutors + numExecutorsToAdd
    +
    +    if (numExecutorsToAdd > 0) {
    +      getCoarseGrainedBackend.foreach { backend =>
    +        logInfo(s"Pending tasks are building up! " +
    +          s"Adding $numExecutorsToAdd new executor(s) (new total is $newNumExecutors).")
    +        numExecutorsPendingToAdd += numExecutorsToAdd
    +        backend.requestExecutors(numExecutorsToAdd)
    +        return numExecutorsToAdd
    +      }
    +    } else {
    +      logDebug(s"Not adding executors because there are already $maxNumExecutors executors, " +
    +        s"which is the limit.")
    +    }
    +    0
    +  }
    +
    +  /**
    +   * Negotiate with the scheduler backend to remove existing executors.
    +   * This ensures the resulting number of executors is correctly constrained by the lower bound.
    +   * Return whether the request to remove the executor is actually sent.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    val numExistingExecutors = executorIds.size - numExecutorsPendingToRemove
    +    if (numExistingExecutors - 1 >= minNumExecutors) {
    +      getCoarseGrainedBackend.foreach { backend =>
    +        logInfo(s"Removing executor $executorId because it has been idle for " +
    +          s"$removeExecutorThreshold seconds (new total is ${numExistingExecutors - 1}).")
    +        numExecutorsPendingToRemove += 1
    +        backend.killExecutor(executorId)
    +        return true
    +      }
    +    } else {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit.")
    +    }
    +    false
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered.")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decrementing pending executors to add (now at $numExecutorsPendingToAdd).")
    +      }
    +      executorIds.add(executorId)
    +      startRemoveExecutorTimer(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      logDebug(s"Existing executor $executorId has been removed.")
    +      executorIds.remove(executorId)
    +      if (numExecutorsPendingToRemove > 0) {
    +        numExecutorsPendingToRemove -= 1
    +        logDebug(s"Decrementing pending executors to remove (now at $numExecutorsPendingToRemove).")
    +      }
    +    } else {
    +      logWarning(s"Not removing unknown executor $executorId")
    --- End diff --
    
    This is a little bit cryptic


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60870919
  
      [Test build #22423 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22423/consoleFull) for   PR 2746 at commit [`a6a30f2`](https://github.com/apache/spark/commit/a6a30f2a0c5adcb5e9af253bf155870e0fa41f71).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60823170
  
    In my mind there are sort of two modes. One is that the user wants to set the exact number of executors and the other is that they want to set a range of executor sizes and have elasticity.
    
    I would actually reject users that try to set both the absolute number and a min or max, since it is not obvious what the semantics should be (as @vanzin points out)
    
    So you'd have like this:
    
    ```
    // Request a fixed number of executors
    ./bin/spark-submit --num-executors = 10
    // Request a dynamically sized cluster
    ./bin/spark-submit --min-executors=5 --max-executor=15
    // INVALID
    ./bin/spark-submit --num-executors 10 --max-executors 15
    ```
    It seems to me a like a type mismatch to accept both "num-executors" and "max-executors" in a single invocation.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59956545
  
    > Even for longer-running tasks, we still have a problem. If the add timer expires multiple times while these tasks are being run, we may end up double counting the number of executors needed.
    
    My assumption was that we would keep the "target" number of executors equal to or less than the number of tasks within the currently running stages, which would solve this problem, IIUC.
    
    That said, I'll stop arguing for Kay's approach.  The exponential approach sounds very reasonable to me as well.
    
    What I do feel really strongly about is that we shouldn't add new configuration knobs that are required to get decent utilization for the average app.  Most of the users I've spoken to find a config as simple as `--executor-cores` difficult to reason about.  And even if they completely grasp how to set it, it's just one other thing they have to think about that distracts them from the job they're actually trying to complete.  This is one of the core complaints form Alex Rubinsteyn's infamous [blog post](http://blog.explainmydata.com/2014/05/spark-should-be-better-than-mapreduce.html).  I think some simple rules could remove this burden from the user.  For example, we could cap the number of outstanding executor requests at the number that would be able to handle all the pending tasks - there's no reason it should ever need to exceed this.  A simple way to explain this to a user would be that Spark will never try to acquire more resources than it would need to run all the work it'
 s ready to run at this moment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59308632
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21787/consoleFull) for   PR 2746 at commit [`44f1832`](https://github.com/apache/spark/commit/44f1832e5be49a7c6f215a318e18c4721650a075).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922413
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    --- End diff --
    
    I'd go for it, especially after reading more of the code for this, since
    you sometimes use millis and sometimes use seconds
    
    On Wed, Oct 15, 2014 at 1:59 PM, andrewor14 <no...@github.com>
    wrote:
    
    > In
    > core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala:
    >
    > > + * request to add or remove executors. The mechanism to actually do this will be added separately,
    > > + * e.g. in SPARK-3822 for Yarn.
    > > + */
    > > +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    > > +  private val conf = scheduler.conf
    > > +
    > > +  // Lower and upper bounds on the number of executors. These are required.
    > > +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    > > +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    > > +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    > > +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    > > +  }
    > > +
    > > +  // How frequently to add and remove executors (seconds)
    > > +  private val addThreshold =
    > > +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    >
    > Yeah this is a broader issue I have filed at
    > https://issues.apache.org/jira/browse/SPARK-3859. I thought about adding
    > Seconds to the end but it makes the name really long. I'm still on the
    > fence about that.
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/2746/files#r18922277>.
    >


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19516815
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,425 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  verifyBounds()
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Verify that the lower and upper bounds on the number of executors are valid.
    +   * If not, throw an appropriate exception.
    +   */
    +  private def verifyBounds(): Unit = {
    +    if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +    }
    +    if (minNumExecutors > maxNumExecutors) {
    +      throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
    +        s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
    +    }
    +  }
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +    // TODO: start at `maxNumExecutors` once SPARK-3822 goes in
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          try {
    +            maybeAddAndRemove()
    +          } catch {
    +            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * If the add time has expired, request new executors and refresh the add time.
    +   * If the remove time for an existing executor has expired, kill the executor.
    +   * This is factored out into its own method for testing.
    +   */
    +  private def maybeAddAndRemove(now: Long = System.currentTimeMillis): Unit = synchronized {
    +    if (addTime != NOT_SET && now >= addTime) {
    +      addExecutors()
    +      logDebug(s"Starting timer to add more executors (to " +
    +        s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +      addTime += sustainedSchedulerBacklogTimeout * 1000
    +    }
    +
    +    removeTimes.foreach { case (executorId, expireTime) =>
    +      if (now >= expireTime) {
    +        removeExecutor(executorId)
    +        removeTimes.remove(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested.
    +   */
    +  private def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already ${executorIds.size} " +
    +        s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
    +        s"tasks are backlogged (new desired total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to remove the given executor.
    +   * Return whether the request is received.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId!")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Attempted to remove executor $executorId " +
    +        s"when it is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left (limit $minNumExecutors)")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  private def onExecutorAdded(
    +      executorId: String,
    +      now: Long = System.currentTimeMillis): Unit = synchronized {
    --- End diff --
    
    Rather than accepting the time all over the place, it's more common to implement a pluggable clock like this:
    ```
    trait Clock {
      def getTime(): Long
    }
    
    class RealClock extends Clock {
      override def getTime() = System.currentTimeMillis
    }
    
    class TestClock extends Clock {
      var curTime: Long = 0
      override def getTime() = curTime
      def tick(ms: Long) = curTime += tick
    }
    
    private[spark] class ExecutorAllocationManager(sc: SparkContext, clock: Clock = new RealClock()) extends Logging {
    ```
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19195109
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPendingToAdd = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // A timestamp of when all pending add requests should expire
    +  private var pendingAddExpirationTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the pending remove request for the executor should expire
    +  private val pendingRemoveExpirationTimes = new mutable.HashMap[String, Long]
    +
    +  // How long before expiring pending requests to add or remove executors (seconds)
    +  private val pendingAddTimeoutSeconds = 300 // 5 min
    +  private val pendingRemoveTimeoutSeconds = 300
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  def initialize(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +
    +              // Expire any outstanding pending add requests that have timed out
    +              if (pendingAddExpirationTime != NOT_STARTED && now >= pendingAddExpirationTime) {
    +                logDebug(s"Expiring all pending add requests because they have " +
    +                  s"not been fulfilled after $pendingAddTimeoutSeconds seconds")
    +                numExecutorsPendingToAdd = 0
    +                pendingAddExpirationTime = NOT_STARTED
    +              }
    +
    +              // Expire any outstanding pending remove requests that have timed out
    +              pendingRemoveExpirationTimes.foreach { case (executorId, expirationTime) =>
    +                if (now > expirationTime) {
    +                  logDebug(s"Expiring pending request to remove executor $executorId because " +
    +                    s"it has not been fulfilled after $pendingRemoveTimeoutSeconds seconds")
    +                  executorsPendingToRemove.remove(executorId)
    +                  pendingRemoveExpirationTimes.remove(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    --- End diff --
    
    Also the word "existing" here is a bit confusing - what does it mean that they exist? It might be more clear to actually keep `execuotrIds.size` and `numExecutorsPendingToAdd` as separate variables. It will make the math below easier to understand.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60477159
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22207/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60991995
  
      [Test build #22461 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22461/consoleFull) for   PR 2746 at commit [`c79e907`](https://github.com/apache/spark/commit/c79e9079014776c28f041bfffdfaf4833dcae7e6).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19557900
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,453 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  verifyBounds()
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  // Clock used to schedule when executors should be added and removed
    +  private var clock: Clock = new RealClock
    +
    +  /**
    +   * Verify that the lower and upper bounds on the number of executors are valid.
    +   * If not, throw an appropriate exception.
    +   */
    +  private def verifyBounds(): Unit = {
    +    if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +    }
    +    if (minNumExecutors == 0 || maxNumExecutors == 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!")
    +    }
    +    if (minNumExecutors > maxNumExecutors) {
    +      throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
    +        s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
    +    }
    +  }
    +
    +  /**
    +   * Use a different clock for this allocation manager. This is mainly used for testing.
    +   */
    +  def setClock(newClock: Clock): Unit = {
    +    clock = newClock
    +  }
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +    // TODO: start at `maxNumExecutors` once SPARK-3822 goes in
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          try {
    +            schedule()
    +          } catch {
    +            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * If the add time has expired, request new executors and refresh the add time.
    +   * If the remove time for an existing executor has expired, kill the executor.
    +   * This is factored out into its own method for testing.
    +   */
    +  private def schedule(): Unit = synchronized {
    +    val now = clock.getTime
    +    if (addTime != NOT_SET && now >= addTime) {
    +      addExecutors()
    +      logDebug(s"Starting timer to add more executors (to " +
    +        s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +      addTime += sustainedSchedulerBacklogTimeout * 1000
    +    }
    +
    +    removeTimes.foreach { case (executorId, expireTime) =>
    +      if (now >= expireTime) {
    +        removeExecutor(executorId)
    +        removeTimes.remove(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested.
    +   */
    +  private def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already ${executorIds.size} " +
    +        s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
    +        s"tasks are backlogged (new desired total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to remove the given executor.
    +   * Return whether the request is received.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId!")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Attempted to remove executor $executorId " +
    +        s"when it is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left (limit $minNumExecutors)")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  private def onExecutorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(onExecutorIdle)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented number of pending executors ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  private def onExecutorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Executor $executorId is no longer pending to " +
    +          s"be removed (${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler receives new pending tasks.
    +   * This sets a time in the future that decides when executors should be added
    +   * if it is not already set.
    +   */
    +  private def onSchedulerBacklogged(): Unit = synchronized {
    +    if (addTime == NOT_SET) {
    +      logDebug(s"Starting timer to add executors because pending tasks " +
    +        s"are building up (to expire in $schedulerBacklogTimeout seconds)")
    +      addTime = clock.getTime + schedulerBacklogTimeout * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler queue is drained.
    +   * This resets all variables used for adding executors.
    +   */
    +  private def onSchedulerQueueEmpty(): Unit = synchronized {
    +    logDebug(s"Clearing timer to add executors because there are no more pending tasks")
    +    addTime = NOT_SET
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is no longer running any tasks.
    +   * This sets a time in the future that decides when this executor should be removed if
    +   * the executor is not already marked as idle.
    +   */
    +  private def onExecutorIdle(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
    +        s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = clock.getTime + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is now running a task.
    +   * This resets all variables used for removing this executor.
    +   */
    +  private def onExecutorBusy(executorId: String): Unit = synchronized {
    +    logDebug(s"Clearing idle timer for $executorId because it is now running a task")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /**
    +   * A listener that notifies the given allocation manager of when to add and remove executors.
    +   *
    +   * This class is intentionally conservative in its assumptions about the relative ordering
    +   * and consistency of events returned by the listener. For simplicity, it does not account
    +   * for speculated tasks.
    +   */
    +  private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +    extends SparkListener {
    +
    +    private val stageIdToNumTasks = new mutable.HashMap[Int, Int]
    +    private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +    private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +    override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
    +      synchronized {
    +        val stageId = stageSubmitted.stageInfo.stageId
    +        val numTasks = stageSubmitted.stageInfo.numTasks
    +        stageIdToNumTasks(stageId) = numTasks
    +        allocationManager.onSchedulerBacklogged()
    +      }
    +    }
    +
    +    override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
    +      synchronized {
    +        val stageId = stageCompleted.stageInfo.stageId
    +        stageIdToNumTasks -= stageId
    +        stageIdToTaskIndices -= stageId
    +
    +        // If this is the last stage with pending tasks, mark the scheduler queue as empty
    +        // This is needed in case the stage is aborted for any reason
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +    }
    +
    +    override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {
    +      val stageId = taskStart.stageId
    +      val taskId = taskStart.taskInfo.taskId
    +      val taskIndex = taskStart.taskInfo.index
    +      val executorId = taskStart.taskInfo.executorId
    +
    +      // If this is the last pending task, mark the scheduler queue as empty
    +      stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex
    +      val numTasksScheduled = stageIdToTaskIndices(stageId).size
    +      val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1)
    +      if (numTasksScheduled == numTasksTotal) {
    +        // No more pending tasks for this stage
    +        stageIdToNumTasks -= stageId
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +
    +      // Mark the executor on which this task is scheduled as busy
    +      executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId
    +      allocationManager.onExecutorBusy(executorId)
    +    }
    +
    +    override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
    +      val executorId = taskEnd.taskInfo.executorId
    +      val taskId = taskEnd.taskInfo.taskId
    +
    +      // If the executor is no longer running scheduled any tasks, mark it as idle
    +      if (executorIdToTaskIds.contains(executorId)) {
    +        executorIdToTaskIds(executorId) -= taskId
    +        if (executorIdToTaskIds(executorId).isEmpty) {
    +          executorIdToTaskIds -= executorId
    +          allocationManager.onExecutorIdle(executorId)
    +        }
    +      }
    +    }
    +
    +    override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = {
    +      val executorId = blockManagerAdded.blockManagerId.executorId
    +      if (executorId != "<driver>") {
    +        allocationManager.onExecutorAdded(executorId)
    +      }
    +    }
    +
    +    override def onBlockManagerRemoved(
    +        blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = {
    +      allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId)
    +    }
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  val NOT_SET = Long.MaxValue
    +}
    +
    +private trait Clock {
    +  def getTime: Long
    +}
    +
    +private class RealClock extends Clock {
    +  override def getTime: Long = System.currentTimeMillis
    +}
    +
    +/**
    + * A clock that allows the caller to customize the time.
    + * This is used mainly for testing.
    + */
    +private class TestClock(startTimeMillis: Long) extends Clock {
    --- End diff --
    
    No action needed on your part, but I wonder if all these test-related classes should start being moved to the `test/` subdir at some point... this is not the only one that exists.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922657
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    --- End diff --
    
    The main reason for factoring this out here is for synchronization. If we rely on the corresponding map (`executorIdToHost`) in the `TaskSchedulerImpl` for this, then we need to synchronize on every usage there as well. In this PR I wanted to make the minimally invasive change to the scheduler code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61000269
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60869227
  
      [Test build #22416 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22416/consoleFull) for   PR 2746 at commit [`b00b680`](https://github.com/apache/spark/commit/b00b680639fbb73778fe2eb28ec54330c99b9d72).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19451757
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    --- End diff --
    
    @pwendell What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19440248
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -0,0 +1,407 @@
    +/*
    + * 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
    +
    +import org.scalatest.FunSuite
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.scheduler._
    +import org.apache.spark.storage.BlockManagerId
    +
    +class ExecutorAllocationManagerSuite extends FunSuite {
    --- End diff --
    
    <3 the huge number of added tests here!!!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377683
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    --- End diff --
    
    I think the wording here should be stronger. I would say that they are not synchronous, but they will be fulfilled eventually if the cluster manager has sufficient resources.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59143131
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21750/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58737472
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21616/Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60992002
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22461/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61024058
  
    Whoops I accidentally merged #3002 into this patch. I'll force push a fix here shortly...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18920780
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    --- End diff --
    
    It would be helpful to expand on what "pending" means in the comment here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60041418
  
    YARN provides (2).  After an app makes a request for X executors, YARN keeps track of them until the app goes away.  In fact, the API to YARN isn't "I want X more executors", but "I want X executors".  I.e. a new request overwrites the old ones (it's a little more convoluted when requests are for particular locations, but basically the same concept).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59132886
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21747/consoleFull) for   PR 2746 at commit [`9bcc0bc`](https://github.com/apache/spark/commit/9bcc0bca1156ce086462ceba162c93310224ed1e).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19507405
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if the time then has exceeded any of the
    +   * add and remove times that are set. If so, it triggers the corresponding action.
    +   */
    +  private def startPolling(): Unit = {
    --- End diff --
    
    It wouldn't silently die; there's a try catch that logs any exception that it faces.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61025071
  
      [Test build #22490 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22490/consoleFull) for   PR 2746 at commit [`8a4fdaa`](https://github.com/apache/spark/commit/8a4fdaad14c546f90acffef274355d884f7e6a59).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59857927
  
    I see that's a good point @andrewor14 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-59137052
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21753/consoleFull) for   PR 2746 at commit [`baaa403`](https://github.com/apache/spark/commit/baaa403b16d10107abeeb1fba721b2c301d2c0aa).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19426761
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number of executors actually requested. Exposed for testing.
    +   */
    +  def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   * Return whether the executor is actually requested to be removed. Exposed for testing.
    +   */
    +  def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left, which is less than the limit $minNumExecutors")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added. Exposed for testing.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed. Exposed for testing.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be
    +   * triggered in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in
    +   * every round thereafter. This is called when the scheduler receives new pending tasks.
    +   * Return the value of the add timer. Exposed for testing.
    +   */
    +  def startAddTimer(): Long = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +    addTime
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   * Return the value of the remove timer. Exposed for testing.
    +   */
    +  def startRemoveTimer(executorId: String): Long = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +    removeTimes(executorId)
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left. Exposed for testing.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task. Exposed for testing.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    +    logDebug(s"Canceling remove executor timer for $executorId")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /* --------------------------- *
    +   | Getters exposed for testing |
    +   * --------------------------- */
    +
    +  def getNumExecutorsToAdd: Int = numExecutorsToAdd
    +  def getNumExecutorsPending: Int = numExecutorsPending
    +  def getExecutorsPendingToRemove: collection.Set[String] = executorsPendingToRemove
    +  def getExecutorIds: collection.Set[String] = executorIds
    +  def getAddTime: Long = addTime
    +  def getRemoveTimes: collection.Map[String, Long] = removeTimes
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  private[spark] val NOT_STARTED = -1L
    --- End diff --
    
    Yeah probably not. It was `private val` before and since I had to use it in tests I relaxed it to `private[spark]`. Maybe it doesn't need to be private anything.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58736135
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21616/consoleFull) for   PR 2746 at commit [`b3c7d44`](https://github.com/apache/spark/commit/b3c7d446160747b79e6afbd844f9c8b6d0158781).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18924277
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -110,19 +110,26 @@ private[spark] class TaskSetManager(
       // the one that it was launched from, but gets removed from them later.
       private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
    +  // List 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.
    +  // List of pending tasks for each rack -- similar to the above.
       private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
     
    -  // Set containing pending tasks with no locality preferences.
    +  // List of pending tasks with no locality preferences.
       var pendingTasksWithNoPrefs = new ArrayBuffer[Int]
     
    -  // Set containing all pending tasks (also used as a stack, as above).
    +  // List of all pending tasks (also used as a stack, as above).
       val allPendingTasks = new ArrayBuffer[Int]
     
    +  // Set of pending tasks used to keep track of whether more executors are needed
    --- End diff --
    
    But can you just update allPendingTasks in the same way you update pendingTasks?  That seems like it doesn't hurt the existing functionality of allPendingTasks, which is updated lazily just for convenience, and in fact would *very slightly* optimize performance, since findTasks wouldn't need to deal with tasks in allPendingTasks that have actually already been scheduled.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18931117
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logInfo(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    // Start the retry timer in case this addition fails
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total is $newTotalExecutors)")
    +    numExecutorsToAdd *= 2
    +    numExecutorsPendingToAdd += actualNumExecutorsToAdd
    +    backend.requestExecutors(actualNumExecutorsToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Retry a previous executor request that has not been fulfilled.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryAddExecutors(): Unit = synchronized {
    +    // Do not retry if there are no executors pending to be added (should never happen)
    +    if (numExecutorsPendingToAdd == 0) {
    +      logWarning("Attempted to retry adding executors when there are none pending to be added")
    +      cancelAddRetryTimer()
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    addRetryAttempts += 1
    +    if (addRetryAttempts > maxAddRetryAttempts) {
    +      logInfo(s"Giving up on adding $numExecutorsPendingToAdd executor(s) " +
    +        s"after $maxAddRetryAttempts failed attempts")
    +      numExecutorsPendingToAdd = 0
    +      // Also cancel original add timer because the cluster is not granting us new executors
    +      cancelAddTimer()
    +      return
    +    }
    +
    +    // Retry a previous request, then restart the retry timer in case this retry also fails
    +    logInfo(s"Previously requested executors have not all registered yet. " +
    +      s"Retrying to add $numExecutorsPendingToAdd executor(s) " +
    +      s"[attempt $addRetryAttempts/$maxAddRetryAttempts]")
    +    backend.requestExecutors(numExecutorsPendingToAdd)
    +    startAddRetryTimer()
    +  }
    +
    +  /**
    +   * Request the scheduler backend to decommission the given executor.
    +   * This expires the remove timer unless the executor is kept alive intentionally.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId) ||
    +        removeRetryAttempts.contains(executorId) ||
    +        retryRemoveTimers.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      cancelRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logDebug(s"Not removing idle executor $executorId because there are only $minNumExecutors " +
    +        "executor(s) left, which is the limit")
    +      // Restart the remove timer to keep the executor marked as idle
    +      // Otherwise we won't be able to remove this executor even after new executors have joined
    +      startRemoveTimer(executorId)
    +      return
    +    }
    +
    +    // Send a kill request to the backend for this executor
    +    // Start the retry timer in case this removal fails
    +    logInfo(s"Removing executor $executorId because it has been idle for " +
    +      s"$removeThreshold seconds (new total is ${numExistingExecutors - 1})")
    +    executorsPendingToRemove.add(executorId)
    +    backend.killExecutor(executorId)
    +    cancelRemoveTimer(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Retry a previous attempt to decommission the given executor.
    +   * This restarts the retry timer to keep trying up to a maximum number of attempts.
    +   */
    +  private def retryRemoveExecutors(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to retry removing unknown executor $executorId")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if this executor is not pending to be killed (should never happen)
    +    if (!executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Attempted to retry removing executor $executorId when it's not to be removed!")
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Do not retry if we have already exceeded the maximum number of attempts
    +    removeRetryAttempts(executorId) =
    +      removeRetryAttempts.getOrElse(executorId, 0) + 1
    +    if (removeRetryAttempts(executorId) > maxRemoveRetryAttempts) {
    +      logInfo(s"Giving up on removing executor $executorId after " +
    +        s"$maxRemoveRetryAttempts failed attempts!")
    +      executorsPendingToRemove.remove(executorId)
    +      cancelRemoveRetryTimer(executorId)
    +      return
    +    }
    +
    +    // Retry a previous kill request for this executor
    +    // Restart the retry timer in case this retry also fails
    +    logInfo(s"Retrying previous attempt to remove $executorId " +
    +      s"[attempt ${removeRetryAttempts(executorId)}/$maxRemoveRetryAttempts]")
    +    backend.killExecutor(executorId)
    +    startRemoveRetryTimer(executorId)
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      logDebug(s"New executor $executorId has registered")
    +      if (numExecutorsPendingToAdd > 0) {
    +        numExecutorsPendingToAdd -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPendingToAdd left)")
    +        if (numExecutorsPendingToAdd == 0) {
    +          logDebug("All previously pending executors have registered!")
    +          cancelAddRetryTimer()
    +        }
    +      }
    +      executorIds.add(executorId)
    +      startRemoveTimer(executorId)
    +    }
    +  }
    +
    +  /**
    +   * Callback for the scheduler to signal that the given executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      logDebug(s"Existing executor $executorId has been removed")
    +      executorIds.remove(executorId)
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +        cancelRemoveRetryTimer(executorId)
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Return whether the add timer is already running.
    +   */
    +  def isAddTimerRunning: Boolean = addTimerEnabled || addRetryTimerEnabled
    +
    +  /**
    +   * Return whether the remove timer for the given executor is already running.
    +   */
    +  def isRemoveTimerRunning(executorId: String): Boolean = {
    +    removeTimers.contains(executorId) || retryRemoveTimers.contains(executorId)
    +  }
    +
    +  /**
    +   * Start a timer to add executors, to expire in `addThreshold` seconds in the first
    +   * round, and `addInterval` seconds in every round thereafter. This is called when
    +   * the scheduler receives new pending tasks and the timer is not already started. This resets
    +   * the value of any existing add timer.
    +   */
    +  def startAddTimer(): Unit = {
    +    val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +    logDebug(s"Starting add executor timer (to expire in $threshold seconds)")
    +    addTimer = 0
    +    addTimerEnabled = true
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor, to expire in `removeThreshold` seconds.
    +   * This is called when an executor registers or finishes running tasks, and the timer is not
    +   * already started. This resets the value of any existing timer to remove this executor.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = {
    +    logDebug(s"Starting remove executor timer for $executorId " +
    +      s"(to expire in $removeThreshold seconds)")
    +    removeTimers(executorId) = 0
    --- End diff --
    
    Shouldn't this be set to `removeThreshold`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60040484
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22020/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19351838
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,313 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    --- End diff --
    
    Sounds reasonable. These are just tentative values that haven't been given a ton of thought yet


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61013959
  
      [Test build #22478 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22478/consoleFull) for   PR 2746 at commit [`c0becc4`](https://github.com/apache/spark/commit/c0becc47611d0e0bbe4c630ab8ab4c52463db616).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String)`
      * `  case class RequestExecutors(requestedTotal: Int) extends CoarseGrainedClusterMessage`
      * `  case class KillExecutors(executorIds: Seq[String]) extends CoarseGrainedClusterMessage`
      * `class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19428912
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    --- End diff --
    
    Ok, I think elsewhere in Spark we don't do this as well. @kayousterhout your suggestion is overridden.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18922176
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,496 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * Both add and remove attempts are retried on failure up to a maximum number of times.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + *   spark.dynamicAllocation.addExecutorRetryInterval - How often to retry adding executors
    + *   spark.dynamicAllocation.removeExecutorRetryInterval - How often to retry removing executors
    + *   spark.dynamicAllocation.maxAddExecutorRetryAttempts - Max retries in re-adding executors
    + *   spark.dynamicAllocation.maxRemoveExecutorRetryAttempts - Max retries in re-removing executors
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors have been removed, both of which
    + * are relatively rare events with respect to task scheduling. Thus, synchronizing each method on
    + * the same lock should not be expensive assuming biased locking is enabled in the JVM (on by
    + * default for Java 6+). This may not be true, however, if the application itself runs multiple
    + * jobs concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorAllocationManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60)
    +  private val addInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addThreshold)
    +  private val addRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorRetryInterval", addInterval)
    +  private val removeThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 600)
    +  private val removeRetryInterval =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorRetryInterval", 300)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // Retry attempts
    +  private var addRetryAttempts = 0
    +  private val removeRetryAttempts = new mutable.HashMap[String, Int]
    +  private val maxAddRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxAddExecutorRetryAttempts", 10)
    +  private val maxRemoveRetryAttempts =
    +    conf.getInt("spark.dynamicAllocation.maxRemoveExecutorRetryAttempts", 10)
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // Timers for keeping track of when to add/remove executors (ms)
    +  private var addTimer = 0
    +  private var addRetryTimer = 0
    +  private val removeTimers = new mutable.HashMap[String, Long]
    +  private val retryRemoveTimers = new mutable.HashMap[String, Long]
    +
    +  // Additional variables used for adding executors
    +  private var addThresholdCrossed = false
    +  private var addTimerEnabled = false
    +  private var addRetryTimerEnabled = false
    +
    +  // Loop interval (ms)
    +  private val intervalMs = 100
    +
    +  // Scheduler backend through which requests to add/remove executors are made
    +  // Note that this assumes the backend has already initialized when this is first used
    +  // Otherwise, an appropriate exception is thrown
    +  private lazy val backend = scheduler.backend match {
    +    case b: CoarseGrainedSchedulerBackend => b
    +    case null =>
    +      throw new SparkException("Scheduler backend not initialized yet!")
    +    case _ =>
    +      throw new SparkException(
    +        "Dynamic allocation of executors is not applicable to fine-grained schedulers. " +
    +        "Please set spark.dynamicAllocation.enabled to false.")
    +  }
    +
    +  initialize()
    +
    +  private def initialize(): Unit = {
    +    // Keep track of all known executors
    +    scheduler.executorIdToHost.keys.foreach(executorAdded)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each interval, this thread checks if any of the timers have expired, and, if
    +   * so, triggers the relevant timer actions.
    +   */
    +  def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run() {
    +        while (true) {
    +          try {
    +            if (addTimerEnabled) {
    +              val threshold = if (addThresholdCrossed) addInterval else addThreshold
    +              if (addTimer > threshold * 1000) {
    +                addThresholdCrossed = true
    +                addExecutors()
    +              }
    +            }
    +
    +            if (addRetryTimerEnabled) {
    +              if (addRetryTimer > addRetryInterval * 1000) {
    +                retryAddExecutors()
    +              }
    +            }
    +
    +            removeTimers.foreach { case (id, t) =>
    +              if (t > removeThreshold * 1000) {
    +                removeExecutor(id)
    +              }
    +            }
    +
    +            retryRemoveTimers.foreach { case (id, t) =>
    +              if (t > removeRetryInterval * 1000) {
    +                retryRemoveExecutors(id)
    +              }
    +            }
    +          } catch {
    +            case e: Exception =>
    +              logError("Exception encountered in dynamic executor allocation thread!", e)
    +          } finally {
    +            // Advance all timers that are enabled
    +            Thread.sleep(intervalMs)
    +            if (addTimerEnabled) {
    +              addTimer += intervalMs
    +            }
    +            if (addRetryTimerEnabled) {
    +              addRetryTimer += intervalMs
    +            }
    +            removeTimers.foreach { case (id, _) =>
    +              removeTimers(id) += intervalMs
    +            }
    +            retryRemoveTimers.foreach { case (id, _) =>
    +              retryRemoveTimers(id) += intervalMs
    +            }
    +          }
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the scheduler backend.
    +   * This automatically restarts the add timer unless it is explicitly canceled.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Restart add timer because there are still pending tasks
    +    startAddTimer()
    +
    +    // Wait until the previous round of executors have registered
    +    if (numExecutorsPendingToAdd > 0) {
    +      logInfo(s"Not adding executors because there are still " +
    +        s"$numExecutorsPendingToAdd request(s) in flight")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPendingToAdd
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logInfo(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    // Start the retry timer in case this addition fails
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +      s"new executor(s) (new total is $newTotalExecutors)")
    +    numExecutorsToAdd *= 2
    +    numExecutorsPendingToAdd += actualNumExecutorsToAdd
    --- End diff --
    
    small nit: above you check that `numExecutorsPendingToAdd` should be zero, otherwise you never reach this code. So this can just be an assignment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61029099
  
      [Test build #22487 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22487/consoleFull) for   PR 2746 at commit [`e7f9684`](https://github.com/apache/spark/commit/e7f9684ec934a00b98fc1d72abd15fbd859ce8e1).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r18738931
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorScalingManager.scala ---
    @@ -0,0 +1,324 @@
    +/*
    + * 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.scheduler
    +
    +import java.util.{Timer, TimerTask}
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.{Logging, SparkException}
    +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
    +
    +/**
    + * An agent that dynamically scales the number of executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks has not
    + * been drained for N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle, meaning it has not been scheduled
    + * to run any tasks, for K seconds, then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * The relevant Spark properties include the following:
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *   spark.dynamicAllocation.addExecutorThreshold - How long before new executors are added (N)
    + *   spark.dynamicAllocation.addExecutorInterval - How often to add new executors (M)
    + *   spark.dynamicAllocation.removeExecutorThreshold - How long before an executor is removed (K)
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention only arises
    + * if the application itself runs multiple jobs concurrently. Under normal circumstances, however,
    + * synchronizing each method on this class should not be expensive assuming biased locking is
    + * enabled in the JVM (on by default for Java 6+). Tighter locks are also used where possible.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[scheduler] class ExecutorScalingManager(scheduler: TaskSchedulerImpl) extends Logging {
    +  private val conf = scheduler.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors
    +  private val addExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThreshold", 60) // s
    +  private val addExecutorInterval =
    +    conf.getLong("spark.dynamicAllocation.addExecutorInterval", addExecutorThreshold) // s
    +  private val removeExecutorThreshold =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThreshold", 300) // s
    +
    +  // Timers that keep track of when to add and remove executors
    +  private var addExecutorTimer: Option[Timer] = None
    +  private val removeExecutorTimers: mutable.Map[String, Timer] = new mutable.HashMap[String, Timer]
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // The number of pending executors that have not actually been added/removed yet
    +  private var numExecutorsPendingToAdd = 0
    +  private var numExecutorsPendingToRemove = 0
    +
    +  // Keep track of all executors here to decouple us from the logic in TaskSchedulerImpl
    +  private val executorIds = new mutable.HashSet[String] ++= scheduler.executorIdToHost.keys
    +
    +  // Start idle timer for all new executors
    +  synchronized { executorIds.foreach(startRemoveExecutorTimer) }
    +
    +  /**
    +   * Start the add executor timer if it does not already exist.
    +   * This is called when a new pending task is added. The add is then triggered
    +   * if the pending tasks queue is not drained in `addExecutorThreshold` seconds.
    +   */
    +  def startAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorThreshold)
    +
    +  /**
    +   * Restart the add executor timer.
    +   * This is called when the previous add executor timer has expired but not canceled. The add
    +   * is then triggered again if all pending executors from the previous round have registered,
    +   * and the pending tasks queue is still not drained in `addExecutorInterval` seconds.
    +   */
    +  private def restartAddExecutorTimer(): Unit = startAddExecutorTimer(addExecutorInterval)
    +
    +  /**
    +   * Start the add executor timer using the given delay if the timer does not already exist.
    +   */
    +  private def startAddExecutorTimer(timerDelaySeconds: Long): Unit = {
    --- End diff --
    
    I'm following the guidelines here http://docs.scala-lang.org/style/declarations.html#procedure-syntax. We've started to enforce this through code reviews in other PRs. Maybe we should clearly document this in the Spark style guide too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19452054
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    --- End diff --
    
    I would echo back the actual values - in some cases users might be confused where the configurations are coming from.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19440348
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    --- End diff --
    
    @pwendell is there a reason not to include units?  I'd argue they should all include units...I'm certainly not a major user and still have spent too long searching the docs to try to figure out what the units for something are.  I think the tiny amount of additional time to type a longer parameter name has a huge benefit in terms of readability down the road?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19454814
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,413 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +  if (minNumExecutors > maxNumExecutors) {
    +    throw new SparkException("spark.dynamicAllocation.minExecutors must " +
    +      "be less than or equal to spark.dynamicAllocation.maxExecutors!")
    +  }
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if the time then has exceeded any of the
    +   * add and remove times that are set. If so, it triggers the corresponding action.
    +   */
    +  private def startPolling(): Unit = {
    --- End diff --
    
    I would refactor things slightly to make this more testable. The main changes I would do is to use a pluggable clock rather than calling `System.currentTimeMillis`. The second thing I would do is move the logic here out into a function called `computeState`. Then I would use an executor service here to schedule invocations of that function.
    
    http://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ScheduledExecutorService.html
    
    Also, it would be good to understand the behavior if an exception is thrown here. At present it seems like it will simply silently die. I would instead log an error with the exception and say that elastic scaling has failed.
    
    Once you've mocked out the clock it will be easier to test things in a nice way. Right now you have `Thread.sleep` in the tests (evil) and it really limits the amount of testing we can do. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19560825
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,453 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If
    + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue
    + * persists for another M seconds, then more executors are added and so on. The number added
    + * in each round increases exponentially from the previous round until an upper bound on the
    + * number of executors has been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
    + * been scheduled to run any tasks, then it is removed.
    + *
    + * There is no retry logic in either case because we make the assumption that the cluster manager
    + * will eventually fulfill all requests it receives asynchronously.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.schedulerBacklogTimeout (M) -
    + *     If there are backlogged tasks for this duration, add new executors
    + *
    + *   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) -
    + *     If the backlog is sustained for this duration, add more executors
    + *     This is used only after the initial backlog timeout is exceeded
    + *
    + *   spark.dynamicAllocation.executorIdleTimeout (K) -
    + *     If an executor has been idle for this duration, remove it
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  verifyBounds()
    +
    +  // How long there must be backlogged tasks for before an addition is triggered
    +  private val schedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.schedulerBacklogTimeout", 60)
    +
    +  // Same as above, but used only after `schedulerBacklogTimeout` is exceeded
    +  private val sustainedSchedulerBacklogTimeout = conf.getLong(
    +    "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout)
    +
    +  // How long an executor must be idle for before it is removed
    +  private val removeThresholdSeconds = conf.getLong(
    +    "spark.dynamicAllocation.executorIdleTimeout", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when an addition should be triggered, or NOT_SET if it is not set
    +  // This is set when pending tasks are added but not scheduled yet
    +  private var addTime: Long = NOT_SET
    +
    +  // A timestamp for each executor of when the executor should be removed, indexed by the ID
    +  // This is set when an executor is no longer running a task, or when it first registers
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis: Long = 100
    +
    +  // Clock used to schedule when executors should be added and removed
    +  private var clock: Clock = new RealClock
    +
    +  /**
    +   * Verify that the lower and upper bounds on the number of executors are valid.
    +   * If not, throw an appropriate exception.
    +   */
    +  private def verifyBounds(): Unit = {
    +    if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +    }
    +    if (minNumExecutors == 0 || maxNumExecutors == 0) {
    +      throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!")
    +    }
    +    if (minNumExecutors > maxNumExecutors) {
    +      throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " +
    +        s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!")
    +    }
    +  }
    +
    +  /**
    +   * Use a different clock for this allocation manager. This is mainly used for testing.
    +   */
    +  def setClock(newClock: Clock): Unit = {
    +    clock = newClock
    +  }
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +    // TODO: start at `maxNumExecutors` once SPARK-3822 goes in
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   */
    +  private def startPolling(): Unit = {
    +    val t = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          try {
    +            schedule()
    +          } catch {
    +            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    t.setName("spark-dynamic-executor-allocation")
    +    t.setDaemon(true)
    +    t.start()
    +  }
    +
    +  /**
    +   * If the add time has expired, request new executors and refresh the add time.
    +   * If the remove time for an existing executor has expired, kill the executor.
    +   * This is factored out into its own method for testing.
    +   */
    +  private def schedule(): Unit = synchronized {
    +    val now = clock.getTime
    +    if (addTime != NOT_SET && now >= addTime) {
    +      addExecutors()
    +      logDebug(s"Starting timer to add more executors (to " +
    +        s"expire in $sustainedSchedulerBacklogTimeout seconds)")
    +      addTime += sustainedSchedulerBacklogTimeout * 1000
    +    }
    +
    +    removeTimes.foreach { case (executorId, expireTime) =>
    +      if (now >= expireTime) {
    +        removeExecutor(executorId)
    +        removeTimes.remove(executorId)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   * Return the number actually requested.
    +   */
    +  private def addExecutors(): Int = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already ${executorIds.size} " +
    +        s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)")
    +      numExecutorsToAdd = 1
    +      return 0
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
    +        numExecutorsToAdd
    +      } else {
    +        maxNumExecutors - numExistingExecutors
    +      }
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " +
    +        s"tasks are backlogged (new desired total will be $newTotalExecutors)")
    +      numExecutorsToAdd =
    +        if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1
    +      numExecutorsPending += actualNumExecutorsToAdd
    +      actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +      0
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to remove the given executor.
    +   * Return whether the request is received.
    +   */
    +  private def removeExecutor(executorId: String): Boolean = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId!")
    +      return false
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Attempted to remove executor $executorId " +
    +        s"when it is already pending to be removed!")
    +      return false
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are only " +
    +        s"$numExistingExecutors executor(s) left (limit $minNumExecutors)")
    +      return false
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +      true
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +      false
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  private def onExecutorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(onExecutorIdle)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented number of pending executors ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  private def onExecutorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      removeTimes.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Executor $executorId is no longer pending to " +
    +          s"be removed (${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler receives new pending tasks.
    +   * This sets a time in the future that decides when executors should be added
    +   * if it is not already set.
    +   */
    +  private def onSchedulerBacklogged(): Unit = synchronized {
    +    if (addTime == NOT_SET) {
    +      logDebug(s"Starting timer to add executors because pending tasks " +
    +        s"are building up (to expire in $schedulerBacklogTimeout seconds)")
    +      addTime = clock.getTime + schedulerBacklogTimeout * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the scheduler queue is drained.
    +   * This resets all variables used for adding executors.
    +   */
    +  private def onSchedulerQueueEmpty(): Unit = synchronized {
    +    logDebug(s"Clearing timer to add executors because there are no more pending tasks")
    +    addTime = NOT_SET
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is no longer running any tasks.
    +   * This sets a time in the future that decides when this executor should be removed if
    +   * the executor is not already marked as idle.
    +   */
    +  private def onExecutorIdle(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) {
    +      logDebug(s"Starting idle timer for $executorId because there are no more tasks " +
    +        s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = clock.getTime + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor is now running a task.
    +   * This resets all variables used for removing this executor.
    +   */
    +  private def onExecutorBusy(executorId: String): Unit = synchronized {
    +    logDebug(s"Clearing idle timer for $executorId because it is now running a task")
    +    removeTimes.remove(executorId)
    +  }
    +
    +  /**
    +   * A listener that notifies the given allocation manager of when to add and remove executors.
    +   *
    +   * This class is intentionally conservative in its assumptions about the relative ordering
    +   * and consistency of events returned by the listener. For simplicity, it does not account
    +   * for speculated tasks.
    +   */
    +  private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager)
    +    extends SparkListener {
    +
    +    private val stageIdToNumTasks = new mutable.HashMap[Int, Int]
    +    private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]]
    +    private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]]
    +
    +    override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
    +      synchronized {
    +        val stageId = stageSubmitted.stageInfo.stageId
    +        val numTasks = stageSubmitted.stageInfo.numTasks
    +        stageIdToNumTasks(stageId) = numTasks
    +        allocationManager.onSchedulerBacklogged()
    +      }
    +    }
    +
    +    override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
    +      synchronized {
    +        val stageId = stageCompleted.stageInfo.stageId
    +        stageIdToNumTasks -= stageId
    +        stageIdToTaskIndices -= stageId
    +
    +        // If this is the last stage with pending tasks, mark the scheduler queue as empty
    +        // This is needed in case the stage is aborted for any reason
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +    }
    +
    +    override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {
    +      val stageId = taskStart.stageId
    +      val taskId = taskStart.taskInfo.taskId
    +      val taskIndex = taskStart.taskInfo.index
    +      val executorId = taskStart.taskInfo.executorId
    +
    +      // If this is the last pending task, mark the scheduler queue as empty
    +      stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex
    +      val numTasksScheduled = stageIdToTaskIndices(stageId).size
    +      val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1)
    +      if (numTasksScheduled == numTasksTotal) {
    +        // No more pending tasks for this stage
    +        stageIdToNumTasks -= stageId
    +        if (stageIdToNumTasks.isEmpty) {
    +          allocationManager.onSchedulerQueueEmpty()
    +        }
    +      }
    +
    +      // Mark the executor on which this task is scheduled as busy
    +      executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId
    +      allocationManager.onExecutorBusy(executorId)
    +    }
    +
    +    override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
    +      val executorId = taskEnd.taskInfo.executorId
    +      val taskId = taskEnd.taskInfo.taskId
    +
    +      // If the executor is no longer running scheduled any tasks, mark it as idle
    +      if (executorIdToTaskIds.contains(executorId)) {
    +        executorIdToTaskIds(executorId) -= taskId
    +        if (executorIdToTaskIds(executorId).isEmpty) {
    +          executorIdToTaskIds -= executorId
    +          allocationManager.onExecutorIdle(executorId)
    +        }
    +      }
    +    }
    +
    +    override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = {
    +      val executorId = blockManagerAdded.blockManagerId.executorId
    +      if (executorId != "<driver>") {
    +        allocationManager.onExecutorAdded(executorId)
    +      }
    +    }
    +
    +    override def onBlockManagerRemoved(
    +        blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = {
    +      allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId)
    +    }
    +  }
    +
    +}
    +
    +private object ExecutorAllocationManager {
    +  val NOT_SET = Long.MaxValue
    +}
    +
    +private trait Clock {
    +  def getTime: Long
    +}
    +
    +private class RealClock extends Clock {
    +  override def getTime: Long = System.currentTimeMillis
    --- End diff --
    
    Ok, I was worried about overflow but I just did some math to find that this is basically impossible:
    ```
    > System.nanoTime
    res0: Long = 1414606728388783000
    > Math.log(Long.MaxValue - System.nanoTime) / Math.log(2)
    res1: Double = 62.75980010864823 // we have 2^62+ nano seconds to spare
    > Math.pow(2, 62) / Math.pow(10, 9) / 86400 / 365
    res2: Double = 146.235604338768
    ```
    This means the application must run for 146+ years before this could happen. I think we can safely assume this won't be the case. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60339555
  
    Alright, as of the latest commit the scheduler uses the suggested interface to request executors. In particular, now it sets an absolute number of pending executors desired rather than an incremental number of additional executors to add. This allows us to remove some timeout logic for pending requests, but only with the assumption that the messages are reliably delivered. I will make the appropriate changes in #2840 as well.
    
    I haven't addressed the minor comments yet. I intend to do so once we settle on a high level design.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377751
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    +  private val addIntervalSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorIntervalSeconds", addThresholdSeconds)
    +  private val removeThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.removeExecutorThresholdSeconds", 600)
    +
    +  // Number of executors to add in the next round
    +  private var numExecutorsToAdd = 1
    +
    +  // Number of executors that have been requested but have not registered yet
    +  private var numExecutorsPending = 0
    +
    +  // Executors that have been requested to be removed but have not been killed yet
    +  private val executorsPendingToRemove = new mutable.HashSet[String]
    +
    +  // All known executors
    +  private val executorIds = new mutable.HashSet[String]
    +
    +  // A timestamp of when the add timer should be triggered, or NOT_STARTED if the timer is not
    +  // started. This timer is started when there are pending tasks built up, and canceled when
    +  // there are no more pending tasks.
    +  private var addTime = NOT_STARTED
    +
    +  // A timestamp for each executor of when the remove timer for that executor should be triggered.
    +  // Each remove timer is started when the executor first registers or when the executor finishes
    +  // running a task, and canceled when the executor is scheduled to run a new task.
    +  private val removeTimes = new mutable.HashMap[String, Long]
    +
    +  // Polling loop interval (ms)
    +  private val intervalMillis = 100
    +
    +  /**
    +   * Register for scheduler callbacks to decide when to add and remove executors.
    +   */
    +  def start(): Unit = {
    +    val listener = new ExecutorAllocationListener(this)
    +    sc.addSparkListener(listener)
    +    startPolling()
    +  }
    +
    +  /**
    +   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * During each loop interval, this thread checks if any of the timers have timed out, and,
    +   * if so, triggers the relevant timer actions.
    +   */
    +  private def startPolling(): Unit = {
    +    val thread = new Thread {
    +      override def run(): Unit = {
    +        while (true) {
    +          ExecutorAllocationManager.this.synchronized {
    +            val now = System.currentTimeMillis
    +            try {
    +              // If the add timer has timed out, add executors and refresh the timer
    +              if (addTime != NOT_STARTED && now >= addTime) {
    +                addExecutors()
    +                logDebug(s"Restarting add executor timer " +
    +                  s"(to be triggered in $addIntervalSeconds seconds)")
    +                addTime += addIntervalSeconds * 1000
    +              }
    +
    +              // If a remove timer has timed out, remove the executor and cancel the timer
    +              removeTimes.foreach { case (executorId, triggerTime) =>
    +                if (now > triggerTime) {
    +                  removeExecutor(executorId)
    +                  cancelRemoveTimer(executorId)
    +                }
    +              }
    +            } catch {
    +              case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    +            }
    +          }
    +          Thread.sleep(intervalMillis)
    +        }
    +      }
    +    }
    +    thread.setName("spark-dynamic-executor-allocation")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /**
    +   * Request a number of executors from the cluster manager.
    +   * If the cap on the number of executors is reached, give up and reset the
    +   * number of executors to add next round instead of continuing to double it.
    +   */
    +  private def addExecutors(): Unit = synchronized {
    +    // Do not request more executors if we have already reached the upper bound
    +    val numExistingExecutors = executorIds.size + numExecutorsPending
    +    if (numExistingExecutors >= maxNumExecutors) {
    +      logDebug(s"Not adding executors because there are already " +
    +        s"$maxNumExecutors executor(s), which is the limit")
    +      numExecutorsToAdd = 1
    +      return
    +    }
    +
    +    // Request executors with respect to the upper bound
    +    val actualNumExecutorsToAdd =
    +      math.min(numExistingExecutors + numExecutorsToAdd, maxNumExecutors) - numExistingExecutors
    +    val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
    +    // TODO: Actually request executors once SPARK-3822 goes in
    +    val addRequestAcknowledged = true // sc.requestExecutors(actualNumbersToAdd)
    +    if (addRequestAcknowledged) {
    +      logInfo(s"Pending tasks are building up! Adding $actualNumExecutorsToAdd " +
    +        s"new executor(s) (new total will be $newTotalExecutors)")
    +      numExecutorsToAdd *= 2
    +      numExecutorsPending += actualNumExecutorsToAdd
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager " +
    +        s"to request $actualNumExecutorsToAdd executors!")
    +    }
    +  }
    +
    +  /**
    +   * Request the cluster manager to decommission the given executor.
    +   */
    +  private def removeExecutor(executorId: String): Unit = synchronized {
    +    // Do not kill the executor if we are not aware of it (should never happen)
    +    if (!executorIds.contains(executorId)) {
    +      logWarning(s"Attempted to remove unknown executor $executorId")
    +      return
    +    }
    +
    +    // Do not kill the executor again if it is already pending to be killed (should never happen)
    +    if (executorsPendingToRemove.contains(executorId)) {
    +      logWarning(s"Executor $executorId is already pending to be removed!")
    +      return
    +    }
    +
    +    // Do not kill the executor if we have already reached the lower bound
    +    val numExistingExecutors = executorIds.size - executorsPendingToRemove.size
    +    if (numExistingExecutors - 1 < minNumExecutors) {
    +      logInfo(s"Not removing idle executor $executorId because there are " +
    +        s"only $minNumExecutors executor(s) left, which is the limit")
    +      return
    +    }
    +
    +    // Send a request to the backend to kill this executor
    +    // TODO: Actually kill the executor once SPARK-3822 goes in
    +    val removeRequestAcknowledged = true // sc.killExecutor(executorId)
    +    if (removeRequestAcknowledged) {
    +      logInfo(s"Removing executor $executorId because it has been idle for " +
    +        s"$removeThresholdSeconds seconds (new total will be ${numExistingExecutors - 1})")
    +      executorsPendingToRemove.add(executorId)
    +    } else {
    +      logWarning(s"Unable to reach the cluster manager to kill executor $executorId!")
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been added.
    +   */
    +  def executorAdded(executorId: String): Unit = synchronized {
    +    if (!executorIds.contains(executorId)) {
    +      executorIds.add(executorId)
    +      executorIds.foreach(startRemoveTimer)
    +      logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})")
    +      if (numExecutorsPending > 0) {
    +        numExecutorsPending -= 1
    +        logDebug(s"Decremented pending executors to add ($numExecutorsPending left)")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Callback invoked when the specified executor has been removed.
    +   */
    +  def executorRemoved(executorId: String): Unit = synchronized {
    +    if (executorIds.contains(executorId)) {
    +      executorIds.remove(executorId)
    +      logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})")
    +      if (executorsPendingToRemove.contains(executorId)) {
    +        executorsPendingToRemove.remove(executorId)
    +        logDebug(s"Removing executor $executorId from pending executors to remove " +
    +          s"(${executorsPendingToRemove.size} left)")
    +      }
    +    } else {
    +      logWarning(s"Unknown executor $executorId has been removed!")
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to add executors if it is not already started. This timer is to be triggered
    +   * in `addThresholdSeconds` in the first round, and `addIntervalSeconds` in every round
    +   * thereafter. This is called when the scheduler receives new pending tasks.
    +   */
    +  def startAddTimer(): Unit = synchronized {
    +    if (addTime == NOT_STARTED) {
    +      logDebug(s"Starting add executor timer because pending tasks " +
    +        s"are building up (to be triggered in $addThresholdSeconds seconds)")
    +      addTime = System.currentTimeMillis + addThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Start a timer to remove the given executor in `removeThresholdSeconds` if the timer is
    +   * not already started. This is called when an executor registers or finishes running a task.
    +   */
    +  def startRemoveTimer(executorId: String): Unit = synchronized {
    +    if (!removeTimes.contains(executorId)) {
    +      logDebug(s"Starting remove timer for $executorId because there are no tasks " +
    +        s"scheduled to run on the executor (to be triggered in $removeThresholdSeconds seconds)")
    +      removeTimes(executorId) = System.currentTimeMillis + removeThresholdSeconds * 1000
    +    }
    +  }
    +
    +  /**
    +   * Cancel any existing add timer.
    +   * This is called when there are no longer pending tasks left.
    +   */
    +  def cancelAddTimer(): Unit = synchronized {
    +    logDebug(s"Canceling add executor timer")
    +    addTime = NOT_STARTED
    +    numExecutorsToAdd = 1
    +  }
    +
    +  /**
    +   * Cancel any existing remove timer for the given executor.
    +   * This is called when this executor is scheduled a new task.
    +   */
    +  def cancelRemoveTimer(executorId: String): Unit = synchronized {
    --- End diff --
    
    The doc, function name, and log message here seem a little off because this is also called in the case where an executor is actually removed due to a timeout (i.e. not only when it's cancelled). What about calling this "clearRemoveTimer"? In general it's not great to discuss the invocation scenario of a function in the doc, because it can become out of date, as it did here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61001184
  
      [Test build #22470 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22470/consoleFull) for   PR 2746 at commit [`c79e907`](https://github.com/apache/spark/commit/c79e9079014776c28f041bfffdfaf4833dcae7e6).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60547196
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22271/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19377737
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    --- End diff --
    
    I have some different proposals for these names here - ones that are more descriptive. Also, I think we tend to avoid having units in the name btw (e.g. "XSeconds") for other configs. It is a little concerning to me how hard it is to explain these configs... but maybe there is a simpler way. Also, did we ever consider having only a single interval here instead of two different ones?
    
    ```
    // Number of seconds of backlog at which to add executors resources
    spark.dynamicAllocation.backlogTimeout
    // Timeout applied to continued backlog (i.e. once the initial backlogTimeout has been surpassed)
    spark.dynamicAllocation.sustainedBacklogTimeout
    
    // Length of time after which to remove idle executors
    spark.dynamicAllocation.idleExecutorTimeout
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60279252
  
    BTW I'm pretty sure the Spark allocator handles those over-allocation scenarios (see ~L258 in YarnAllocator.scala, `if (numExecutorsRunningNow > maxExecutors) {`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [WIP][SPARK-3795] Heuristics for dynamically s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-58609629
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21568/Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60705067
  
      [Test build #22326 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22326/consoleFull) for   PR 2746 at commit [`abdea61`](https://github.com/apache/spark/commit/abdea61a7d797a822b1cdc539da964134f546ac9).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-60880622
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/2746#issuecomment-61029952
  
    Alright, I tested the latest code on a yarn cluster and verified that my cluster scaled up and down according to the workload as expected. I'm merging this. Thanks everyone for reviewing.
    
    There will be a few follow up patches:
    - SPARK-4134: Tone down scary messages when executors are removed as expected
    - SPARK-4136: Cancel outstanding requests if queue becomes empty
    - SPARK-4138: Guard against incompatible settings on number of executors (#3002)
    - SPARK-4139: Start the number of executors at the max (also #3002)
    - SPARK-4140: Document the configs
    
    The last 3 are expected to block the 1.2 release while the rest are nice-to-haves.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-3795] Heuristics for dynamically scalin...

Posted by sryza <gi...@git.apache.org>.
Github user sryza commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2746#discussion_r19440516
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable
    +
    +import org.apache.spark.scheduler._
    +
    +/**
    + * An agent that dynamically allocates and removes executors based on the workload.
    + *
    + * The add policy depends on the number of pending tasks. If the queue of pending tasks is not
    + * drained in N seconds, then new executors are added. If the queue persists for another M
    + * seconds, then more executors are added and so on. The number added in each round increases
    + * exponentially from the previous round until an upper bound on the number of executors has
    + * been reached.
    + *
    + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly
    + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
    + * we may add more executors than we need just to remove them later. (2) Executors should be added
    + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take
    + * a long time to ramp up under heavy workloads.
    + *
    + * The remove policy is simpler: If an executor has been idle for K seconds (meaning it has not
    + * been scheduled to run any tasks), then it is removed. This requires starting a timer on each
    + * executor instead of just starting a global one as in the add case.
    + *
    + * There is no retry logic in either case. Because the requests to the cluster manager are
    + * asynchronous, this class does not know whether a request has been granted until later. For
    + * this reason, both add and remove are treated as best-effort only.
    + *
    + * The relevant Spark properties include the following:
    + *
    + *   spark.dynamicAllocation.enabled - Whether this feature is enabled
    + *   spark.dynamicAllocation.minExecutors - Lower bound on the number of executors
    + *   spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors
    + *
    + *   spark.dynamicAllocation.addExecutorThresholdSeconds - How long before new executors are added
    + *   spark.dynamicAllocation.addExecutorIntervalSeconds - How often to add new executors
    + *   spark.dynamicAllocation.removeExecutorThresholdSeconds - How long before an executor is removed
    + *
    + * Synchronization: Because the schedulers in Spark are single-threaded, contention should only
    + * arise when new executors register or when existing executors are removed, both of which are
    + * relatively rare events with respect to task scheduling. Thus, synchronizing each method on the
    + * same lock should not be expensive assuming biased locking is enabled in the JVM (on by default
    + * for Java 6+). This may not be true, however, if the application itself runs multiple jobs
    + * concurrently.
    + *
    + * Note: This is part of a larger implementation (SPARK-3174) and currently does not actually
    + * request to add or remove executors. The mechanism to actually do this will be added separately,
    + * e.g. in SPARK-3822 for Yarn.
    + */
    +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging {
    +  import ExecutorAllocationManager._
    +
    +  private val conf = sc.conf
    +
    +  // Lower and upper bounds on the number of executors. These are required.
    +  private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1)
    +  private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1)
    +  if (minNumExecutors < 0 || maxNumExecutors < 0) {
    +    throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!")
    +  }
    +
    +  // How frequently to add and remove executors (seconds)
    +  private val addThresholdSeconds =
    +    conf.getLong("spark.dynamicAllocation.addExecutorThresholdSeconds", 60)
    --- End diff --
    
    In general I'm a fan of including units, but I think starting to include them now could be confusing, given that none of the existing configuration properties have them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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