Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19041#discussion_r165210078
  
    --- Diff: core/src/main/scala/org/apache/spark/CacheRecoveryManager.scala 
---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.{ConcurrentHashMap, ScheduledFuture, TimeUnit}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.util.{Success => Succ}
    +import scala.util.Failure
    +
    +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, RDDBlockId}
    +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: mutable.Set[String] =
    +    ConcurrentHashMap.newKeySet[String]().asScala
    +
    +  /**
    +   * Start the recover cache shutdown process for these executors
    +   *
    +   * @param execIds the executors to start shutting down
    +   */
    +  def startCacheRecovery(execIds: Seq[String]): Unit = {
    +    logDebug(s"Recover cached data before shutting down executors 
${execIds.mkString(", ")}.")
    +    val canBeRecovered = checkMem(execIds)
    +    recoveringExecutors ++= canBeRecovered
    +    val executorsWithKillTimers = 
canBeRecovered.zip(canBeRecovered.map(startKillTimer))
    +    executorsWithKillTimers.foreach((replicateUntilDone _).tupled)
    +  }
    +
    +  /**
    +   * 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.
    +   *
    +   * @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 }
    --- End diff --
    
    conversely, you might decide to not even try to replicate some executors, 
because *all* of their data can't be replicated, but in fact *some* of the 
blocks on the executor could be replicated (maybe even the most useful ones).
    
    given the limitations & complications of this method, I'm wondering whether 
its even worth it to do this filtering?


---

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

Reply via email to