You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/15 08:58:57 UTC

[GitHub] [spark] Ngone51 commented on a change in pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

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



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -420,6 +420,21 @@ package object config {
       .booleanConf
       .createWithDefault(false)
 
+  private[spark] val STORAGE_SHUFFLE_DECOMMISSION_ENABLED =
+    ConfigBuilder("spark.storage.decommission.shuffle_blocks")

Review comment:
       According to the new configuration naming policy([DISCUSS-naming-policy-of-Spark-configs](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html#a28878)) , we should append `enabled` for the boolean configuration if it's end with noun.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -489,6 +492,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("Received shuffle index block update for ${shuffleId} ${mapId}, ignoring.")

Review comment:
       nit: missing the leading "s".

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,281 @@
+/*
+ * 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 =>
+              logInfo("Nothing to migrate")

Review comment:
       I'm afraid this log might be annoying when there's no shuffle data to migrate or all the shuffle data has been migrated.

##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -420,6 +420,21 @@ package object config {
       .booleanConf
       .createWithDefault(false)
 
+  private[spark] val STORAGE_SHUFFLE_DECOMMISSION_ENABLED =
+    ConfigBuilder("spark.storage.decommission.shuffle_blocks")

Review comment:
       +1 for avoiding using `_` in config names.

##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1725,6 +1725,17 @@ class SparkContext(config: SparkConf) extends Logging {
     }
   }
 
+
+  private[spark] def decommissionExecutors(executorIds: Seq[String]): Unit = {

Review comment:
       Do you plan to make this a public API in the future? I don't see the usage of it in this PR.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -334,13 +336,14 @@ class BlockManagerMasterEndpoint(
     val info = blockManagerInfo(blockManagerId)
 
     val rddBlocks = info.blocks.keySet().asScala.filter(_.isRDD)
-    rddBlocks.map { blockId =>
+    val result = rddBlocks.map { blockId =>
       val currentBlockLocations = blockLocations.get(blockId)
       val maxReplicas = currentBlockLocations.size + 1
       val remainingLocations = currentBlockLocations.toSeq.filter(bm => bm != blockManagerId)
       val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas)
       replicateMsg
     }.toSeq
+    result

Review comment:
       Is this a necessary change?

##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -121,12 +121,28 @@ private class ShuffleStatus(numPartitions: Int) {
     mapStatuses(mapIndex) = status
   }
 
+  /**
+   * Update the map output location (e.g. during migration).
+   */
+  def updateMapOutput(mapId: Long, bmAddress: BlockManagerId): Unit = withWriteLock {
+    val mapStatusOpt = mapStatuses.find(_.mapId == mapId)
+    mapStatusOpt match {
+      case Some(mapStatus) =>
+        logInfo(s"Updating map output for ${mapId} to ${bmAddress}")
+        mapStatus.updateLocation(bmAddress)

Review comment:
       Shall we add an `assert(mapStatus.location != bmAddress)` here as a sanity check?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,281 @@
+/*
+ * 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 =>
+              logInfo("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
+                logInfo(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, ExecutorService)]()
+
+  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 = {
+      var failures = 0
+      while (!stopped && !Thread.interrupted() && failures < 20) {
+        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")
+          }
+          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" +
+              "spark.storage.decommission.shuffle_blocks\n" +
+              "spark.storage.decommission.rdd_blocks")
+          }
+          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) =>
+            failures += 1
+            logError("Error occurred while trying to replicate cached RDD blocks" +
+              s" for block manager decommissioning (failure count: $failures)", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * 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()
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq
+    shufflesToMigrate.addAll(newShufflesToMigrate.asJava)
+    migratingShuffles ++= newShufflesToMigrate
+
+    // Update the threads doing migrations
+    // TODO: Sort & only start as many threads as min(||blocks||, ||targets||) using location pref
+    val livePeerSet = bm.getPeers(false).toSet
+    val currentPeerSet = migrationPeers.keys.toSet
+    val deadPeers = currentPeerSet.&~(livePeerSet)
+    val newPeers = livePeerSet.&~(currentPeerSet)
+    migrationPeers ++= newPeers.map { peer =>
+      logDebug(s"Starting thread to migrate shuffle blocks to ${peer}")
+      val executor = ThreadUtils.newDaemonSingleThreadExecutor(s"migrate-shuffle-to-${peer}")

Review comment:
       I realize that the `peer` plays as a consumer role in this case, but still worry that this could bump many threads at the same time when there're many available peers, especially for the large cluster.
   
   Is it better to set up a configurable thread pool in this case?
   
   

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,281 @@
+/*
+ * 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 =>
+              logInfo("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
+                logInfo(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, ExecutorService)]()
+
+  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 = {
+      var failures = 0
+      while (!stopped && !Thread.interrupted() && failures < 20) {
+        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")
+          }
+          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" +
+              "spark.storage.decommission.shuffle_blocks\n" +
+              "spark.storage.decommission.rdd_blocks")
+          }
+          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) =>
+            failures += 1
+            logError("Error occurred while trying to replicate cached RDD blocks" +
+              s" for block manager decommissioning (failure count: $failures)", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * 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()
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq
+    shufflesToMigrate.addAll(newShufflesToMigrate.asJava)
+    migratingShuffles ++= newShufflesToMigrate
+
+    // Update the threads doing migrations
+    // TODO: Sort & only start as many threads as min(||blocks||, ||targets||) using location pref
+    val livePeerSet = bm.getPeers(false).toSet
+    val currentPeerSet = migrationPeers.keys.toSet
+    val deadPeers = currentPeerSet.&~(livePeerSet)
+    val newPeers = livePeerSet.&~(currentPeerSet)

Review comment:
       There might be fake live peers if we don't get the latest peers forcibly? I remember the RDD migration always get the latest peers forcibly.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,281 @@
+/*
+ * 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 =>
+              logInfo("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
+                logInfo(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, ExecutorService)]()
+
+  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 = {
+      var failures = 0
+      while (!stopped && !Thread.interrupted() && failures < 20) {
+        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")
+          }
+          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" +
+              "spark.storage.decommission.shuffle_blocks\n" +
+              "spark.storage.decommission.rdd_blocks")
+          }
+          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) =>
+            failures += 1
+            logError("Error occurred while trying to replicate cached RDD blocks" +
+              s" for block manager decommissioning (failure count: $failures)", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * 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()
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq
+    shufflesToMigrate.addAll(newShufflesToMigrate.asJava)
+    migratingShuffles ++= newShufflesToMigrate
+
+    // Update the threads doing migrations
+    // TODO: Sort & only start as many threads as min(||blocks||, ||targets||) using location pref
+    val livePeerSet = bm.getPeers(false).toSet
+    val currentPeerSet = migrationPeers.keys.toSet
+    val deadPeers = currentPeerSet.&~(livePeerSet)
+    val newPeers = livePeerSet.&~(currentPeerSet)
+    migrationPeers ++= newPeers.map { peer =>
+      logDebug(s"Starting thread to migrate shuffle blocks to ${peer}")
+      val executor = ThreadUtils.newDaemonSingleThreadExecutor(s"migrate-shuffle-to-${peer}")
+      val runnable = new ShuffleMigrationRunnable(peer)
+      executor.submit(runnable)
+      (peer, (runnable, executor))
+    }
+    // A peer may have entered a decommissioning state, don't transfer any new blocks
+    deadPeers.foreach { peer =>
+        migrationPeers.get(peer).foreach(_._1.running = false)

Review comment:
       We should also add the migrating ShuffleBlockInfo back to the queue in this case? Otherwise, the ShuffleBlockInfo which is under migrating may be missing if there's no exception is thrown. 

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,281 @@
+/*
+ * 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 =>
+              logInfo("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
+                logInfo(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, ExecutorService)]()
+
+  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 = {
+      var failures = 0
+      while (!stopped && !Thread.interrupted() && failures < 20) {
+        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")
+          }
+          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" +
+              "spark.storage.decommission.shuffle_blocks\n" +
+              "spark.storage.decommission.rdd_blocks")

Review comment:
       Shouldn't we stop decommission immediately if no tasks are configured?




----------------------------------------------------------------
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:
users@infra.apache.org



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