agrawaldevesh commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r443318618



##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -157,7 +158,8 @@ class BlockManagerMasterEndpoint(
       context.reply(true)
 
     case DecommissionBlockManagers(executorIds) =>

Review comment:
       I wanted to confirm my understanding that this PR does not do anything 
about shuffle files written by executors that have since finished ? That would 
be the External Shuffle Service's responsibility (if enabled), Right ? And that 
piece of the puzzle isn't implemented yet ?
   
   My question stems from how we handle this scenario: (Consider the standalone 
scheduler mode here)
   - Executor wrote some shuffle data and is enabled with external shuffle 
service 
   - Executor finished normally and nothing is running on the worker. 
   - DecommissionExecutor is not even called because the executor does not exist
   - Thus the DecommissionBlockManager isn't called and thus no migrations take 
place.
   
   Therefore fetch failures would happen for other tasks that try to read this 
executor's shuffle data (via the External Shuffle Service). I believe fixing 
this is not in the scope of this PR.
   
   Can you please check this understanding. Thanks.
   

##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -489,6 +491,24 @@ class BlockManagerMasterEndpoint(
       storageLevel: StorageLevel,
       memSize: Long,
       diskSize: Long): Boolean = {
+    logInfo(s"Updating block info on master ${blockId} for ${blockManagerId}")
+
+    if (blockId.isInternalShuffle) {
+      blockId match {
+        case ShuffleIndexBlockId(shuffleId, mapId, _) =>
+          // Don't update the map output on just the index block
+          logDebug(s"Received shuffle index block update for ${shuffleId} 
${mapId}, ignoring.")
+          return true
+        case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) =>
+          logInfo(s"Received shuffle data block update for ${shuffleId} 
${mapId}, updating.")

Review comment:
       Should this be a logDebug ?

##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.storage
+
+import java.util.concurrent.ExecutorService
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config
+import org.apache.spark.shuffle.{MigratableResolver, ShuffleBlockInfo}
+import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Class to handle block manager decommissioning retries.
+ * It creates a Thread to retry offloading all RDD cache and Shuffle blocks
+ */
+private[storage] class BlockManagerDecommissioner(
+  conf: SparkConf,
+  bm: BlockManager) extends Logging {
+
+  private val maxReplicationFailuresForDecommission =
+    conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK)
+
+  /**
+   * This runnable consumes any shuffle blocks in the queue for migration. 
This part of a
+   * producer/consumer where the main migration loop updates the queue of 
blocks to be migrated
+   * periodically. On migration failure, the current thread will reinsert the 
block for another
+   * thread to consume. Each thread migrates blocks to a different particular 
executor to avoid
+   * distribute the blocks as quickly as possible without overwhelming any 
particular executor.
+   *
+   * There is no preference for which peer a given block is migrated to.
+   * This is notable different than the RDD cache block migration (further 
down in this file)
+   * which uses the existing priority mechanism for determining where to 
replicate blocks to.
+   * Generally speaking cache blocks are less impactful as they normally 
represent narrow
+   * transformations and we normally have less cache present than shuffle data.
+   *
+   * The producer/consumer model is chosen for shuffle block migration to 
maximize
+   * the chance of migrating all shuffle blocks before the executor is forced 
to exit.
+   */
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends 
Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[ShuffleBlockInfo] = None
+      logInfo(s"Starting migration thread for ${peer}")
+      // Once a block fails to transfer to an executor stop trying to transfer 
more blocks
+      try {
+        while (running && !Thread.interrupted()) {
+          val migrating = Option(shufflesToMigrate.poll())
+          migrating match {
+            case None =>
+              logDebug("Nothing to migrate")
+              // Nothing to do right now, but maybe a transfer will fail or a 
new block
+              // will finish being committed.
+              val SLEEP_TIME_SECS = 1
+              Thread.sleep(SLEEP_TIME_SECS * 1000L)
+            case Some(shuffleBlockInfo) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleBlockInfo} to 
${peer}")
+              val blocks =
+                bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                bm.blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  StorageLevel.DISK_ONLY,
+                  null)// class tag, we don't need for shuffle
+                logDebug(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}")
+          }
+        }
+        // This catch is intentionally outside of the while running block.
+        // if we encounter errors migrating to an executor we want to stop.
+      } catch {
+        case e: Exception =>
+          migrating match {
+            case Some(shuffleMap) =>
+              logError(s"Error ${e} during migration, adding ${shuffleMap} 
back to migration queue")
+              shufflesToMigrate.add(shuffleMap)
+            case None =>
+              logError(s"Error ${e} while waiting for block to migrate")
+          }
+      }
+    }
+  }
+
+  // Shuffles which are either in queue for migrations or migrated
+  private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]()
+
+  // Shuffles which are queued for migration
+  private[storage] val shufflesToMigrate =
+    new java.util.concurrent.ConcurrentLinkedQueue[ShuffleBlockInfo]()
+
+  @volatile private var stopped = false
+
+  private val migrationPeers =
+    mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  private lazy val blockMigrationExecutor =
+    ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission")
+
+  private val blockMigrationRunnable = new Runnable {
+    val sleepInterval = 
conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL)
+
+    override def run(): Unit = {
+      if (!conf.get(config.STORAGE_RDD_DECOMMISSION_ENABLED) &&
+        !conf.get(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED)) {
+        logWarning("Decommissioning, but no task configured set one or 
both:\n" +
+          s"${config.STORAGE_RDD_DECOMMISSION_ENABLED.key}\n" +
+          s"${config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED.key}")
+        stopped = true
+      }
+      while (!stopped && !Thread.interrupted()) {
+        logInfo("Iterating on migrating from the block manager.")
+        try {
+          // If enabled we migrate shuffle blocks first as they are more 
expensive.
+          if (conf.get(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED)) {
+            logDebug("Attempting to replicate all shuffle blocks")
+            offloadShuffleBlocks()
+            logInfo("Done starting workers to migrate shuffle blocks")
+          }
+          if (conf.get(config.STORAGE_RDD_DECOMMISSION_ENABLED)) {
+            logDebug("Attempting to replicate all cached RDD blocks")
+            decommissionRddCacheBlocks()
+            logInfo("Attempt to replicate all cached blocks done")
+          }
+          logInfo(s"Waiting for ${sleepInterval} before refreshing 
migrations.")
+          Thread.sleep(sleepInterval)
+        } catch {
+          case e: InterruptedException =>
+            logInfo("Interrupted during migration, will not refresh 
migrations.")
+            stopped = true
+          case NonFatal(e) =>
+            logError("Error occurred while trying to replicate for block 
manager decommissioning.",
+              e)
+            stopped = true
+        }
+      }
+    }
+  }
+
+  lazy val shuffleMigrationPool = ThreadUtils.newDaemonCachedThreadPool(
+    "migrate-shuffles",
+    conf.get(config.STORAGE_SHUFFLE_DECOMMISSION_MAX_THREADS))
+  /**
+   * Tries to offload all shuffle blocks that are registered with the shuffle 
service locally.
+   * Note: this does not delete the shuffle files in-case there is an 
in-progress fetch
+   * but rather shadows them.
+   * Requires an Indexed based shuffle resolver.
+   * Note: if called in testing please call stopOffloadingShuffleBlocks to 
avoid thread leakage.
+   */
+  private[storage] def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = bm.migratableResolver.getStoredShuffles()

Review comment:
       I am a super n00b to this code and I am not totally sure, but could 
there be a race here: 
   
   org.apache.spark.shuffle.IndexShuffleBlockResolver#writeIndexFileAndCommit 
is called by a task on executor A finishing up its shuffle writing. This 
creates the shuffle files on the disk. But the block manager hasn't yet noticed 
them and it hasn't yet registered them with the block manager master.
   
   But we offload the block in the meanwhile and the peer gets the new block 
and it is able to update the block manager master (and thus the map output 
tracker). Eventually, the source (executor A) also does this registration and 
now the map output tracker is re-updated.
   
   I am not sure if this can happen even. I didn't fully follow the code on how 
we update the BlockManager in the first place :-). In any case, even if it 
does, this whole migration thing is a best effort anyway.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

Reply via email to