Github user brad-kaiser commented on a diff in the pull request:
https://github.com/apache/spark/pull/19041#discussion_r175125932
--- 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
+ * no more blocks
+ */
+ private def replicateUntilDone(execId: String): Future[Boolean] = {
+ recoverLatestBlock(execId).flatMap { moreBlocks =>
+ if (moreBlocks) replicateUntilDone(execId) else Future.successful(false)
+ }
+ }
+
+ /**
+ * Replicate the latest cached rdd block off of this executor on to a surviving executor,
and then
+ * remove the block from this executor
+ *
+ * @param execId the executor to recover a block from
+ * @return A future holding the id of the block that was recovered or None if there
were no blocks
--- End diff --
this matches now
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
|