Github user vanzin commented on a diff in the pull request: https://github.com/apache/spark/pull/19041#discussion_r163045943 --- Diff: core/src/main/scala/org/apache/spark/CacheRecoveryManager.scala --- @@ -0,0 +1,187 @@ +/* + * 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 java.util.concurrent.{ScheduledFuture, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} +import scala.util.Failure + +import com.google.common.cache.CacheBuilder + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.DYN_ALLOCATION_CACHE_RECOVERY_TIMEOUT +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.ThreadUtils + +/** + * Responsible for asynchronously replicating all of an executor's cached blocks, and then shutting + * it down. + */ +final private class CacheRecoveryManager( + blockManagerMasterEndpoint: RpcEndpointRef, + executorAllocationManager: ExecutorAllocationManager, + conf: SparkConf) + extends Logging { + + private val forceKillAfterS = conf.get(DYN_ALLOCATION_CACHE_RECOVERY_TIMEOUT) + private val threadPool = ThreadUtils.newDaemonCachedThreadPool("cache-recovery-manager-pool") + private implicit val asyncExecutionContext: ExecutionContext = + ExecutionContext.fromExecutorService(threadPool) + private val scheduler = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("cache-recovery-shutdown-timers") + private val recoveringExecutors = CacheBuilder.newBuilder() + .expireAfterWrite(forceKillAfterS * 2, TimeUnit.SECONDS) + .build[String, java.lang.Boolean]() // Cache value never used and must extend Object + + /** + * Start the recover cache shutdown process for these executors + * + * @param execIds the executors to start shutting down + * @return a sequence of futures representing the kill process. + */ + def startCacheRecovery(execIds: Seq[String]): Seq[Future[Boolean]] = { + logDebug(s"Recover cached data before shutting down executors ${execIds.mkString(", ")}.") + val canBeRecovered = checkMem(execIds) + canBeRecovered.foreach { execId => recoveringExecutors.put(execId, true)} + canBeRecovered.map { execId => + val killTimer = startKillTimer(execId) + replicateUntilDone(execId) + .andThen { + // don't log if kill timer is done, because it would be redundant + case Failure(e) if !killTimer.isDone => + logWarning(s"Failure recovering cached data before executor $execId shutdown", e) + } + .andThen { + case _ => + if (killTimer.cancel(false)) { kill(execId) } + } + } + } + + /** + * Given a list of executors that will be shut down, check if there is enough free memory on the + * rest of the cluster to hold their data. Return a list of just the executors for which there + * will be enough space. Executors are included smallest first. + * + * This is a best guess implementation and it is not guaranteed that all returned executors + * will succeed. For example a block might be too big to fit on any one specific executor. + * + * @param execIds executors which will be shut down + * @return a Seq of the executors we do have room for + */ + private def checkMem(execIds: Seq[String]): Seq[String] = { + val execsToShutDown = execIds.toSet + // Memory Status is a map of executor Id to a tuple of Max Memory and remaining memory on that + // executor. + val allExecMemStatus: Map[String, (Long, Long)] = blockManagerMasterEndpoint + .askSync[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + .map { case (blockManagerId, mem) => blockManagerId.executorId -> mem } + + val (expiringMemStatus, remainingMemStatus) = allExecMemStatus.partition { + case (execId, _) => execsToShutDown.contains(execId) + } + val freeMemOnRemaining = remainingMemStatus.values.map(_._2).sum + + // The used mem on each executor sorted from least used mem to greatest + val executorAndUsedMem: Seq[(String, Long)] = + expiringMemStatus.map { case (execId, (maxMem, remainingMem)) => + val usedMem = maxMem - remainingMem + execId -> usedMem + }.toSeq.sortBy { case (_, usedMem) => usedMem } + + executorAndUsedMem + .scan(("start", freeMemOnRemaining)) { + case ((_, freeMem), (execId, usedMem)) => (execId, freeMem - usedMem) + } + .drop(1) + .filter { case (_, freeMem) => freeMem > 0 } + .map(_._1) + } + + /** + * Given an executor id, start a timer that will kill the given executor after the configured + * timeout + * + * @param execId The id of the executor to be killed + * @return a future representing the timer + */ + private def startKillTimer(execId: String): ScheduledFuture[_] = { + val killer = new Runnable { + def run(): Unit = { + logDebug(s"Killing $execId because timeout for recovering cached data has expired") + kill(execId) + } + } + scheduler.schedule(killer, forceKillAfterS, TimeUnit.SECONDS) + } + + /** + * Recover cached RDD blocks off of an executor until there are no more, or until + * there is an error + * + * @param execId the id of the executor to be killed + * @return a Future of Boolean that returns true if a block was copied and false if there were --- End diff -- The return value description sounds weird (two different unrelated things for true and false). Maybe just: A future that returns whether there are more blocks to replicate. Although you don't really use that return value outside of this method. So you could just return a `Future[Unit]` here, and handle the boolean internally.
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org