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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to