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/02 19:12:06 UTC

[GitHub] [spark] holdenk opened a new pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

holdenk opened a new pull request #28708:
URL: https://github.com/apache/spark/pull/28708


   ### Why are the changes needed?
   
   Recomputting shuffle blocks can be expensive, we should take advantage of our decommissioning time to migrate these blocks.
   
   ### Does this PR introduce any user-facing change?
   
   This PR introduces two new configs parameters, `spark.storage.decommission.shuffle_blocks` & `spark.storage.decommission.rdd_blocks` that control which blocks should be migrated during storage decommissioning.
   
   ### How was this patch tested?
   
   New unit test & expansion of the Spark on K8s decom test to assert that decommisioning with shuffle block migration means that the results are not recomputed even when the original executor is terminated.
   
   This PR is a cleaned-up version of the previous WIP PR I made https://github.com/apache/spark/pull/28331 (thanks to @attilapiros for his very helpful reviewing on it :)).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
shaneknapp commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660315331


   test this please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658910619


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659726455


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/30618/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638578913






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644954130


   **[Test build #124137 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124137/testReport)** for PR 28708 at commit [`7da5130`](https://github.com/apache/spark/commit/7da5130198e957e0292219f00f04c776ee97da19).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r447249716



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -242,8 +244,7 @@ private[spark] class BlockManager(
 
   private var blockReplicationPolicy: BlockReplicationPolicy = _
 
-  private var blockManagerDecommissioning: Boolean = false
-  private var decommissionManager: Option[BlockManagerDecommissionManager] = None
+  @volatile private var decommissioner: Option[BlockManagerDecommissioner] = None

Review comment:
       That's true. If we drop it we might also accept remove block puts after we've started decommissioning though. Depends on how much we want to avoid that.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644931371


   **[Test build #124136 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124136/testReport)** for PR 28708 at commit [`56c5538`](https://github.com/apache/spark/commit/56c5538700c3ace747c02ce00a531c8d00265863).
    * This patch **fails build dependency tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639007168


   > > So @attilapiros looking at the Jenkins console logs we aren't leaking any threads during testing (nor would I expect us to). But I'll add something to more aggressively stop the shuffle migration threads.
   > 
   > It will come when the `BlockManager` will be tested in `BlockManagerSuite`:
   > 
   > ```
   >  ===== POSSIBLE THREAD LEAK IN SUITE o.a.s.storage.BlockManagerSuite, thread names: rpc-boss-3-1, migrate-shuffle-to-BlockManagerId(exec2, localhost, 50804, None), shuffle-boss-9-1  , shuffle-boss-6-1 =====
   > ```
   
   Gotcha was looking for the explicit decom test. I'll eagerly shutdown the migrate-shuffle-to threads then.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639625978


   Thanks for the PR :)


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658984638


   @jiangxb1987, hasn't there been ample time to review? The majority of the changes have been available to review for a month now. And @holdenk already [paused to allow time for more review](https://github.com/apache/spark/pull/28708#issuecomment-646399281):
   
   > I'm going to hold off on merging this for a little longer actually, it seems like there are some other folks who seem interested in the space . . .


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456115190



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -55,6 +58,24 @@ private[spark] class IndexShuffleBlockResolver(
 
   def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None)
 
+  /**
+   * Get the shuffle files that are stored locally. Used for block migrations.
+   */
+  override def getStoredShuffles(): Set[ShuffleBlockInfo] = {
+    // Matches ShuffleIndexBlockId name
+    val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r

Review comment:
       I did some poking and it seems like we can drop this regex and instead do some pattern matching off of getAllBlocks from the disk manager. Thanks for pointing out this was weird.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643575373


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r447247000



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -55,6 +58,25 @@ private[spark] class IndexShuffleBlockResolver(
 
   def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None)
 
+  /**
+   * Get the shuffle files that are stored locally. Used for block migrations.
+   */
+  override def getStoredShuffles(): Set[ShuffleBlockInfo] = {
+    // Matches ShuffleIndexBlockId name
+    val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r
+    val rootDirs = blockManager.diskBlockManager.localDirs
+    // ExecutorDiskUtil puts things inside one level hashed sub directories
+    val searchDirs = rootDirs.flatMap(_.listFiles()).filter(_.isDirectory()) ++ rootDirs
+    val filenames = searchDirs.flatMap(_.list())
+    logDebug(s"Got block files ${filenames.toList}")
+    filenames.flatMap { fname =>
+      pattern.findAllIn(fname).matchData.map {
+        matched => ShuffleBlockInfo(matched.group(1).toInt, matched.group(2).toLong)
+      }
+    }.toSet

Review comment:
       There shouldn't be any duplicates, but if there are we only need to transfer one anyways.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639202742






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r436071464



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1813,107 @@ private[spark] class BlockManager(
     }
   }
 
+
+  // Shuffles which are either in queue for migrations or migrated
+  private val migratingShuffles = mutable.HashSet[(Int, Long)]()
+  // Shuffles which are queued for migration
+  private val shufflesToMigrate = new java.util.concurrent.ConcurrentLinkedQueue[(Int, Long)]()
+
+
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("Error ${e} during migration, adding ${shuffleMap} back to migration queue")
+              shufflesToMigrate.add(shuffleMap)
+            case None =>
+              logError("Error ${e} while waiting for block to migrate")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()
+    logInfo(s"My local shuffles are ${localShuffles.toList}")
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq
+    logInfo(s"My new shuffles to migrate ${newShufflesToMigrate.toList}")
+    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 = 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)

Review comment:
       I don't think we need that change specifically, right now setting `running` to false seems to do the job.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660397588


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30693/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
prakharjain09 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r455008439



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1285,6 +1314,9 @@ private[spark] class BlockManager(
 
     require(blockId != null, "BlockId is null")
     require(level != null && level.isValid, "StorageLevel is null or invalid")
+    if (isDecommissioning()) {
+      throw new BlockSavedOnDecommissionedBlockManagerException(blockId)

Review comment:
       doPut on a decommissioning executor might haven been called by the running task - so can this line lead to task failure? 




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639095830


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123541/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r454881428



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,330 @@
+/*
+ * 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, Int)] = 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()) {
+          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, retryCount)) =>
+              if (retryCount < maxReplicationFailuresForDecommission) {
+                logInfo(s"Trying to migrate shuffle ${shuffleBlockInfo} to ${peer}")
+                val blocks =
+                  bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo)
+                logDebug(s"Got migration sub-blocks ${blocks}")
+                blocks.foreach { case (blockId, buffer) =>
+                  logDebug(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}")
+              } else {
+                logError(s"Skipping block ${shuffleBlockInfo} because it has failed ${retryCount}")
+              }
+          }
+        }
+        // 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, retryCount)) =>
+              logError(s"Error during migration, adding ${shuffleMap} back to migration queue", e)
+              shufflesToMigrate.add((shuffleMap, retryCount + 1))
+            case None =>
+              logError(s"Error while waiting for block to migrate", e)
+          }
+      }
+    }
+  }
+
+  // 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, Int)]()

Review comment:
       Could you add an explanation for the entry type, especially for the `Int`?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644982889


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28754/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642183054






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637839744






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-650477676


   Just an FYI to folks I'm not as active on this PR as I would normally be as I'm waiting to see where the SPIP discussions go. I'll circle back to this next week.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439386843



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,265 @@
+/*
+ * 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
+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)
+
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = 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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                bm.migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              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 ${shuffleId},${mapId} 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[(Int, Long)]()
+
+  // Shuffles which are queued for migration
+  private[storage] val shufflesToMigrate =
+    new java.util.concurrent.ConcurrentLinkedQueue[(Int, Long)]()
+
+  @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)

Review comment:
       So it tries to migrate to whichever live peer is available first, and if that fails re-inserts the shuffle file into the queue.
   
   The cache block one is different in a few ways: 1) we have a policy for determining what the replication target host is based on the current replication info in the block, 2) we only have to migrate one block at a time (here we have two separate blocks that need to migrate).
   
   I think the producer/consumer model is optimal for the shuffle blocks where we just want to migrate them quickly, but not a good fit for the cache blocks where we have a policy to choose a target block manager for a given block.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r438156385



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -242,8 +244,8 @@ private[spark] class BlockManager(
 
   private var blockReplicationPolicy: BlockReplicationPolicy = _
 
-  private var blockManagerDecommissioning: Boolean = false
-  private var decommissionManager: Option[BlockManagerDecommissionManager] = None
+  @volatile private var blockManagerDecommissioning: Boolean = false

Review comment:
       Nit: Do we need the `blockManagerDecommissioning` flag or would it be enough to test whether `decommissionManager` is defined instead?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643797896


   The K8s integration test failure is irrelevant to this PR.
   ```
   - Run SparkPi with no resources *** FAILED ***
   ```


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-650477142


   Retest this please.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638537610


   **[Test build #123515 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123515/testReport)** for PR 28708 at commit [`ccb8827`](https://github.com/apache/spark/commit/ccb8827c61e3178c83ae505dbddd30b1a6a2d361).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637929460






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639007187


   **[Test build #123539 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123539/testReport)** for PR 28708 at commit [`81e29a8`](https://github.com/apache/spark/commit/81e29a81209b99c113dd8bf71b5887c35392b733).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639055024


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639007990


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659759628


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126005/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660345179


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30686/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-650506061


   **[Test build #124557 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124557/testReport)** for PR 28708 at commit [`ac096f4`](https://github.com/apache/spark/commit/ac096f46aa6e658704d7726efe7d66ece280b83e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-640763276






----------------------------------------------------------------
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


[GitHub] [spark] attilapiros edited a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
attilapiros edited a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-641294356


   Hi @holdenk! 
   
   I have opened a new PR for your branch: https://github.com/holdenk/spark/pull/6
   
   I know this seems to be a lot of change but it is really just moving code out from the `BlockManager`. 
   
   One advantage is testability: now the `BlockManagerDecommissionManager` can be tested quite easily without a `BlockManager`.
   
   The other is now the `BlockManagerDecommissionManager` has very clean interface for its dependencies:
   - `MigratableResolver` to handle Shuffle Blocks 
   - `MigratableRDDResolver` to handle cached RDD
   - `BlockTransferService` for handling block uploads
   - `BlockManagerPeerProvider` to find the peers where to migrate those blocks
   
   I suggest to quickly check the commits and you will see it is not that risky and as refactors on production code is quite discouraged probably this is the best time to make these steps.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642146987


   **[Test build #123773 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123773/testReport)** for PR 28708 at commit [`e4f2fb7`](https://github.com/apache/spark/commit/e4f2fb7b28727ace4a4ca15109e5b26139e16ff8).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645635067






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645635047


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28794/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643560584


   **[Test build #123960 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123960/testReport)** for PR 28708 at commit [`fe34308`](https://github.com/apache/spark/commit/fe34308ae700540559d50094e817f58cb681b402).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r455369019



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,330 @@
+/*
+ * 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,

Review comment:
       nit, these should be 4 space indented

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,330 @@
+/*
+ * 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

Review comment:
       these creates a thread per add and shuffle block migration correct? and possibly another pool for the actual migration. Wonder if we can just clarify or generalize




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439186710



##########
File path: core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.shuffle
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.client.StreamCallbackWithID
+import org.apache.spark.serializer.SerializerManager
+import org.apache.spark.storage.BlockId
+
+/**
+ * :: Experimental ::
+ * An experimental trait to allow Spark to migrate shuffle blocks.
+ */
+@Experimental
+trait MigratableResolver {
+  /**
+   * Get the shuffle ids that are stored locally. Used for block migrations.

Review comment:
       Perhaps the return tuple (int, long) should be a case class for better documentation ?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639004563


   > So @attilapiros looking at the Jenkins console logs we aren't leaking any threads during testing (nor would I expect us to). But I'll add something to more aggressively stop the shuffle migration threads.
   
   It will come when the `BlockManager` will be tested in `BlockManagerSuite`: 
   
   ```
    ===== POSSIBLE THREAD LEAK IN SUITE o.a.s.storage.BlockManagerSuite, thread names: rpc-boss-3-1, migrate-shuffle-to-BlockManagerId(exec2, localhost, 50804, None), shuffle-boss-9-1  , shuffle-boss-6-1 =====
   ``` 


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638540600


   **[Test build #123510 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123510/testReport)** for PR 28708 at commit [`b2da4c5`](https://github.com/apache/spark/commit/b2da4c555154e4c74eac9f97cdf85796bd51748f).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638458440


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r454906637



##########
File path: core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.Semaphore
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+
+import org.scalatest.concurrent.Eventually
+
+import org.apache.spark._
+import org.apache.spark.internal.config
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
+import org.apache.spark.util.{ResetSystemProperties, ThreadUtils}
+
+class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext
+    with ResetSystemProperties with Eventually {
+
+  val numExecs = 3
+  val numParts = 3
+
+  test(s"verify that an already running task which is going to cache data succeeds " +
+    s"on a decommissioned executor") {
+    runDecomTest(true, false, true)
+  }
+
+  test(s"verify that shuffle blocks are migrated") {
+    runDecomTest(false, true, false)
+  }
+
+  test(s"verify that both migrations can work at the same time.") {
+    runDecomTest(true, true, false)
+  }
+
+  private def runDecomTest(persist: Boolean, shuffle: Boolean, migrateDuring: Boolean) = {
+
+    val master = s"local-cluster[${numExecs}, 1, 1024]"
+    val conf = new SparkConf().setAppName("test").setMaster(master)
+      .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist)
+      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle)
+    // Just replicate blocks as fast as we can during testing, there isn't another
+    // workload we need to worry about.
+      .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 1L)
+
+    sc = new SparkContext(master, "test", conf)
+
+    // Wait for the executors to start
+    TestUtils.waitUntilExecutorsUp(sc = sc,
+      numExecutors = numExecs,
+      timeout = 60000) // 60s
+
+    // Create input RDD with 10 partitions
+    val input = sc.parallelize(1 to numParts, numParts)
+    val accum = sc.longAccumulator("mapperRunAccumulator")
+    input.count()
+
+    // Create a new RDD where we have sleep in each partition, we are also increasing
+    // the value of accumulator in each partition
+    val baseRdd = input.mapPartitions { x =>
+      if (migrateDuring) {
+        Thread.sleep(1000)
+      }
+      accum.add(1)
+      x.map(y => (y, y))
+    }
+    val testRdd = shuffle match {
+      case true => baseRdd.reduceByKey(_ + _)
+      case false => baseRdd
+    }
+
+    // Listen for the job & block updates
+    val taskStartSem = new Semaphore(0)
+    val broadcastSem = new Semaphore(0)
+    val executorRemovedSem = new Semaphore(0)
+    val taskEndEvents = ArrayBuffer.empty[SparkListenerTaskEnd]
+    val blocksUpdated = ArrayBuffer.empty[SparkListenerBlockUpdated]
+    sc.addSparkListener(new SparkListener {
+
+      override def onExecutorRemoved(execRemoved: SparkListenerExecutorRemoved): Unit = {
+        executorRemovedSem.release()
+      }
+
+      override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
+        taskStartSem.release()
+      }
+
+      override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
+        taskEndEvents.append(taskEnd)
+      }
+
+      override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = {
+        // Once broadcast start landing on the executors we're good to proceed.
+        // We don't only use task start as it can occur before the work is on the executor.
+        if (blockUpdated.blockUpdatedInfo.blockId.isBroadcast) {
+          broadcastSem.release()
+        }
+        blocksUpdated.append(blockUpdated)
+      }
+    })
+
+
+    // Cache the RDD lazily
+    if (persist) {
+      testRdd.persist()
+    }
+
+    // Start the computation of RDD - this step will also cache the RDD
+    val asyncCount = testRdd.countAsync()
+
+    // Wait for the job to have started.
+    taskStartSem.acquire(1)
+    // Wait for each executor + driver to have it's broadcast info delivered.
+    broadcastSem.acquire((numExecs + 1))
+
+    // Make sure the job is either mid run or otherwise has data to migrate.
+    if (migrateDuring) {
+      // Give Spark a tiny bit to start executing after the broadcast blocks land.
+      // For me this works at 100, set to 300 for system variance.
+      Thread.sleep(300)
+    } else {
+      ThreadUtils.awaitResult(asyncCount, 15.seconds)
+    }
+
+    // Decommission one of the executors.
+    val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend]
+    val execs = sched.getExecutorIds()
+    assert(execs.size == numExecs, s"Expected ${numExecs} executors but found ${execs.size}")
+
+    val execToDecommission = execs.head
+    logDebug(s"Decommissioning executor ${execToDecommission}")
+    sched.decommissionExecutor(execToDecommission)
+
+    // Wait for job to finish.
+    val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 15.seconds)
+    assert(asyncCountResult === numParts)
+    // All tasks finished, so accum should have been increased numParts times.
+    assert(accum.value === numParts)
+
+    sc.listenerBus.waitUntilEmpty()
+    if (shuffle) {
+      //  mappers & reducers which succeeded
+      assert(taskEndEvents.count(_.reason == Success) === 2 * numParts,
+        s"Expected ${2 * numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})")
+    } else {
+      // only mappers which executed successfully
+      assert(taskEndEvents.count(_.reason == Success) === numParts,
+        s"Expected ${numParts} tasks got ${taskEndEvents.size} (${taskEndEvents})")
+    }
+
+    // Wait for our respective blocks to have migrated
+    eventually(timeout(30.seconds), interval(10.milliseconds)) {
+      if (persist) {
+        // One of our blocks should have moved.
+        val rddUpdates = blocksUpdated.filter { update =>
+          val blockId = update.blockUpdatedInfo.blockId
+          blockId.isRDD}
+        val blockLocs = rddUpdates.map { update =>
+          (update.blockUpdatedInfo.blockId.name,
+            update.blockUpdatedInfo.blockManagerId)}
+        val blocksToManagers = blockLocs.groupBy(_._1).mapValues(_.size)
+        assert(!blocksToManagers.filter(_._2 > 1).isEmpty,
+          s"We should have a block that has been on multiple BMs in rdds:\n ${rddUpdates} from:\n" +
+          s"${blocksUpdated}\n but instead we got:\n ${blocksToManagers}")
+      }
+      // If we're migrating shuffles we look for any shuffle block updates
+      // as there is no block update on the initial shuffle block write.
+      if (shuffle) {
+        val numDataLocs = blocksUpdated.filter { update =>
+          val blockId = update.blockUpdatedInfo.blockId
+          blockId.isInstanceOf[ShuffleDataBlockId]
+        }.size
+        val numIndexLocs = blocksUpdated.filter { update =>
+          val blockId = update.blockUpdatedInfo.blockId
+          blockId.isInstanceOf[ShuffleIndexBlockId]
+        }.size
+        assert(numDataLocs >= 1, s"Expect shuffle data block updates in ${blocksUpdated}")

Review comment:
       What's the exact number? numParts? If so, why not use `numParts`?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660353428






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660134481


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660363807


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30689/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456061843



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,330 @@
+/*
+ * 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

Review comment:
       Yeah so I've been thinking about that a bit. The reason for the configurable size pool for shuffle block migrations is shuffle blocks tend to be large and we want to allow the user to control the parallelism to match their network set up & general decommissioning time.
   
   I think having the management threads separate is ok since otherwise, this gets a bit confusing. We could do the same thing with some complicated locking, but I think keeping the tread pools distinct is easier to read.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643501798


   **[Test build #123941 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123941/testReport)** for PR 28708 at commit [`7f00072`](https://github.com/apache/spark/commit/7f00072a93053881cb6ee3f69f3be0560142337e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644982878






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440410286



##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -775,7 +802,12 @@ private[spark] class MapOutputTrackerMaster(
   override def stop(): Unit = {
     mapOutputRequests.offer(PoisonPill)
     threadpool.shutdown()
-    sendTracker(StopMapOutputTracker)
+    try {
+      sendTracker(StopMapOutputTracker)
+    } catch {
+      case e: Exception =>

Review comment:
       Thanks~




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440375111



##########
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'll switch the log level to debug.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637902434


   cc @vanzin, @jiangxb1987, @Ngone51 too FYI


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638541000


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123510/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638550092


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28137/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r455360479



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -55,6 +58,24 @@ private[spark] class IndexShuffleBlockResolver(
 
   def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None)
 
+  /**
+   * Get the shuffle files that are stored locally. Used for block migrations.
+   */
+  override def getStoredShuffles(): Set[ShuffleBlockInfo] = {
+    // Matches ShuffleIndexBlockId name
+    val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r
+    val rootDirs = blockManager.diskBlockManager.localDirs
+    // ExecutorDiskUtil puts things inside one level hashed sub directories
+    val searchDirs = rootDirs.flatMap(_.listFiles()).filter(_.isDirectory()) ++ rootDirs

Review comment:
       this feels like it should live in the diskBlockManager? isn't this close to getAllFiles?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660397596






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658009462






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660411307


   **[Test build #126080 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126080/testReport)** for PR 28708 at commit [`8494bdd`](https://github.com/apache/spark/commit/8494bdd94285c7cc5a41e151da920710be7f4671).
    * This patch **fails PySpark pip packaging tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-640797313






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658965320


   cc @squito , @viirya , @tgravescs 


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660582133


   The python packaging tests are failing on Jenkins post upgrade and this passes all of the GH actions so unless there is any more discussion I intend to merge this tomorrow.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r438269803



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala
##########
@@ -0,0 +1,265 @@
+/*
+ * 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
+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 BlockManagerDecommissionManager(

Review comment:
       Yeah that's a good name




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638916984


   I just found a bug in my `BlockManagerSuite` test which is caused by this line: 
   
   https://github.com/apache/spark/blob/ccb8827c61e3178c83ae505dbddd30b1a6a2d361/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala#L54
   
   During offloading once this should refer to the source block manager and once the target.
   I do not know how it was running yesterday (today I rebased on top of the new changes) successfully but it was green. 
   
   Anyway I would like to answer your last question but for this I needed to do some more testing.
    
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639628299


   Oh also I think I understand some of our disagreement over the threads. I thought you were asking me to stop the Spark executor because I’ve started doing some separate work on that, and you were talking about the migration threads. Is my understanding correct now?


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-651900158


   **[Test build #124671 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124671/testReport)** for PR 28708 at commit [`56a9903`](https://github.com/apache/spark/commit/56a990316a947785b2502e3129c49ab00bef9b90).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660381704


   It looks like the R test is failing even with the upgrade. I'm going to disable it and file a blocker to re-enable it unless folks object to that approach.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638593225


   **[Test build #123515 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123515/testReport)** for PR 28708 at commit [`ccb8827`](https://github.com/apache/spark/commit/ccb8827c61e3178c83ae505dbddd30b1a6a2d361).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440377415



##########
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:
       good point, I'll set stopped to true.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639202742


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644955823


   **[Test build #124137 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124137/testReport)** for PR 28708 at commit [`7da5130`](https://github.com/apache/spark/commit/7da5130198e957e0292219f00f04c776ee97da19).
    * This patch **fails build dependency tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643660661


   **[Test build #123981 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123981/testReport)** for PR 28708 at commit [`da1db47`](https://github.com/apache/spark/commit/da1db4740778b3f12df88e4c28aa0602ff15417e).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638420488


   **[Test build #123499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123499/testReport)** for PR 28708 at commit [`d988e7f`](https://github.com/apache/spark/commit/d988e7fd913495231ca6b841c54943a84822fdd7).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642167745


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28397/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639007969


   **[Test build #123539 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123539/testReport)** for PR 28708 at commit [`81e29a8`](https://github.com/apache/spark/commit/81e29a81209b99c113dd8bf71b5887c35392b733).
    * This patch **fails Scala style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643502482






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643575581


   Yeah so the plan is to trigger an exit as soon as migrations are completed. I think a good follow up to  https://issues.apache.org/jira/browse/SPARK-31197 would be adding a timeout so we also just exit if were likely to be killed anyways by the underlying cluster manager (k8s/yarn/etc.).
   
   Also to make it a bit clearer I've put up two WIP PRs that build on top of this PR: https://github.com/apache/spark/pull/28818 & https://github.com/apache/spark/pull/28817 , there obviously a bit earlier on in terms of implementation/test coverage but I think they help illustrate some of the next steps.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639628456


   **[Test build #123572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123572/testReport)** for PR 28708 at commit [`2ca6c08`](https://github.com/apache/spark/commit/2ca6c08e00b9378365a7f4bd8fccec9f85960b58).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r434896155



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -650,6 +657,19 @@ private[spark] class BlockManager(
       blockId: BlockId,
       level: StorageLevel,
       classTag: ClassTag[_]): StreamCallbackWithID = {
+    // Delegate shuffle blocks here to resolver if supported

Review comment:
       Cool :)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637780196






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658534819


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125867/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659083176


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125905/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657985918


   Jenkins retest this please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658944693


   All checks pass, I'm going to merge this to our current development branch.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440372693



##########
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:
       If it completes without any exception then it is fully migrated and it's the other executors responsibility to migrate it now.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644982878


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660363823






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638512982






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-651930328


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/29287/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644429042






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645620924


   **[Test build #124178 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124178/testReport)** for PR 28708 at commit [`ac096f4`](https://github.com/apache/spark/commit/ac096f46aa6e658704d7726efe7d66ece280b83e).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660411445






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657983486


   **[Test build #125799 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125799/testReport)** for PR 28708 at commit [`5a0cd2a`](https://github.com/apache/spark/commit/5a0cd2abd316aacc601b9e8fa6e1406b67c55fb7).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `public final class MapOutputCommitMessage `
     * `  case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage`
     * `sealed trait LogisticRegressionSummary extends ClassificationSummary `
     * `sealed trait RandomForestClassificationSummary extends ClassificationSummary `
     * `class _ClassificationSummary(JavaWrapper):`
     * `class _TrainingSummary(JavaWrapper):`
     * `class _BinaryClassificationSummary(_ClassificationSummary):`
     * `class LinearSVCModel(_JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable,`
     * `class LinearSVCSummary(_BinaryClassificationSummary):`
     * `class LinearSVCTrainingSummary(LinearSVCSummary, _TrainingSummary):`
     * `class LogisticRegressionSummary(_ClassificationSummary):`
     * `class LogisticRegressionTrainingSummary(LogisticRegressionSummary, _TrainingSummary):`
     * `class BinaryLogisticRegressionSummary(_BinaryClassificationSummary,`
     * `class RandomForestClassificationSummary(_ClassificationSummary):`
     * `class RandomForestClassificationTrainingSummary(RandomForestClassificationSummary,`
     * `class BinaryRandomForestClassificationSummary(_BinaryClassificationSummary):`
     * `class BinaryRandomForestClassificationTrainingSummary(BinaryRandomForestClassificationSummary,`
     * `  class DisableHints(conf: SQLConf) extends RemoveAllHints(conf: SQLConf) `
     * `case class WithFields(`
     * `case class Hour(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField `
     * `case class Minute(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField `
     * `case class Second(child: Expression, timeZoneId: Option[String] = None) extends GetTimeField `
     * `trait GetDateField extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant `
     * `case class DayOfYear(child: Expression) extends GetDateField `
     * `case class SecondsToTimestamp(child: Expression) extends UnaryExpression`
     * `case class Year(child: Expression) extends GetDateField `
     * `case class YearOfWeek(child: Expression) extends GetDateField `
     * `case class Quarter(child: Expression) extends GetDateField `
     * `case class Month(child: Expression) extends GetDateField `
     * `case class DayOfMonth(child: Expression) extends GetDateField `
     * `case class DayOfWeek(child: Expression) extends GetDateField `
     * `case class WeekDay(child: Expression) extends GetDateField `
     * `case class WeekOfYear(child: Expression) extends GetDateField `
     * `sealed trait UTCTimestamp extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant `
     * `case class FromUTCTimestamp(left: Expression, right: Expression) extends UTCTimestamp `
     * `case class ToUTCTimestamp(left: Expression, right: Expression) extends UTCTimestamp `
     * `sealed abstract class MergeAction extends Expression with Unevaluable `
     * `case class DeleteAction(condition: Option[Expression]) extends MergeAction`
     * `trait BaseScriptTransformationExec extends UnaryExecNode `
     * `abstract class BaseScriptTransformationWriterThread(`
     * `abstract class BaseScriptTransformIOSchema extends Serializable `
     * `case class CoalesceBucketsInSortMergeJoin(conf: SQLConf) extends Rule[SparkPlan] `
     * `class StateStoreConf(`
     * `case class HiveScriptTransformationExec(`


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642183063


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28397/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435555809



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} to ${peer}")

Review comment:
       We don't delete the file from the current host right away. Once the BlockUpdate message is processed on the master it will go to the peer it has been migrated to.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440443890



##########
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) {

Review comment:
       Revisiting this as I was writing the docs for the config I think we can remove it, we've got all of the other reasonable failures to retry covered elsewhere.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639055033


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28164/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644954130


   **[Test build #124137 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124137/testReport)** for PR 28708 at commit [`7da5130`](https://github.com/apache/spark/commit/7da5130198e957e0292219f00f04c776ee97da19).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642183054


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-640797313






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637896250


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] rdblue edited a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
rdblue edited a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658984638


   @jiangxb1987, hasn't there been ample time to review? The majority of the changes have been available to review for a month now. And @holdenk already [paused to allow time for more review](https://github.com/apache/spark/pull/28708#issuecomment-646399281) nearly 4 weeks ago:
   
   > I'm going to hold off on merging this for a little longer actually, it seems like there are some other folks who seem interested in the space . . .


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r454916426



##########
File path: core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
##########
@@ -1866,13 +1903,57 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
     assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId))
     assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId))
 
-    store1.decommissionRddCacheBlocks()
+    val decomManager = new BlockManagerDecommissioner(conf, store1)
+    decomManager.decommissionRddCacheBlocks()
     // Smaller block offloaded to store2
     assert(master.getLocations(blockIdSmall) === Seq(store2.blockManagerId))
     // Larger block still present in store1 as it can't be offloaded
     assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId))
   }
 
+  test("test migration of shuffle blocks during decommissioning") {
+    val shuffleManager1 = makeSortShuffleManager()
+    val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1)
+    shuffleManager1.shuffleBlockResolver._blockManager = bm1
+
+    val shuffleManager2 = makeSortShuffleManager()
+    val bm2 = makeBlockManager(3500, "exec2", shuffleManager = shuffleManager2)
+    shuffleManager2.shuffleBlockResolver._blockManager = bm2
+
+    val blockSize = 5
+    val shuffleDataBlockContent = Array[Byte](0, 1, 2, 3, 4)
+    val shuffleData = ShuffleDataBlockId(0, 0, 0)
+    Files.write(bm1.diskBlockManager.getFile(shuffleData).toPath(), shuffleDataBlockContent)
+    val shuffleIndexBlockContent = Array[Byte](5, 6, 7, 8, 9)
+    val shuffleIndex = ShuffleIndexBlockId(0, 0, 0)
+    Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent)
+
+    mapOutputTracker.registerShuffle(0, 1)
+    val decomManager = new BlockManagerDecommissioner(conf, bm1)
+    try {
+      mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0))
+      assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId)
+
+      val env = mock(classOf[SparkEnv])
+      when(env.conf).thenReturn(conf)
+      SparkEnv.set(env)
+
+      decomManager.refreshOffloadingShuffleBlocks()
+
+      eventually(timeout(1.second), interval(10.milliseconds)) {
+        assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId)

Review comment:
       Maybe it's better to remove shuffle data in `bm1` before checking `bm2`?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r438152775



##########
File path: core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 scala.concurrent.duration._
+
+import org.mockito.{ArgumentMatchers => mc}
+import org.mockito.Mockito.{mock, times, verify, when}
+import org.scalatest._
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark._
+import org.apache.spark.internal.config
+import org.apache.spark.network.BlockTransferService
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.shuffle.MigratableResolver
+import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock
+
+class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers {
+
+  private val bmPort = 12345
+
+  private val sparkConf = new SparkConf(false)
+    .set(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED, true)
+    .set(config.STORAGE_RDD_DECOMMISSION_ENABLED, true)
+
+  private def registerShuffleBlocks(
+      mockMigratableShuffleResolver: MigratableResolver,
+      ids: Set[(Int, Long, Int)]): Unit = {
+
+    when(mockMigratableShuffleResolver.getStoredShuffles())
+      .thenReturn(ids.map(triple => (triple._1, triple._2)).toSet)
+
+    ids.foreach { case (shuffleId: Int, mapId: Long, reduceId: Int) =>
+      when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any(), mc.any()))
+        .thenReturn(List(
+          (ShuffleIndexBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])),
+          (ShuffleDataBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer]))))
+    }
+  }
+
+  test("test shuffle and cached rdd migration without any error") {
+    val blockTransferService = mock(classOf[BlockTransferService])
+    val bm = mock(classOf[BlockManager])
+
+    val storedBlockId1 = RDDBlockId(0, 0)
+    val storedBlock1 =
+      new ReplicateBlock(storedBlockId1, Seq(BlockManagerId("replicaHolder", "host1", bmPort)), 1)
+
+    val migratableShuffleBlockResolver = mock(classOf[MigratableResolver])
+    registerShuffleBlocks(migratableShuffleBlockResolver, Set((1, 1L, 1)))
+    when(bm.getPeers(mc.any()))
+      .thenReturn(Seq(BlockManagerId("exec2", "host2", 12345)))
+
+    when(bm.blockTransferService).thenReturn(blockTransferService)
+    when(bm.migratableResolver).thenReturn(migratableShuffleBlockResolver)
+    when(bm.getMigratableRDDBlocks())
+      .thenReturn(Seq(storedBlock1))
+
+    val bmDecomManager = new BlockManagerDecommissionManager(

Review comment:
       Nit: one line




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644454930


   @holdenk The PR description is not clear. The current PR description does not have the first section in the template "What changes were proposed in this pull request?" . Could you please update it and explain how you did it? 


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644955844


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/124137/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644463707






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643562280


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28580/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439853387



##########
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")

Review comment:
       Could you describe a little more about which root cause do you have in mind?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-652069482


   **[Test build #124671 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124671/testReport)** for PR 28708 at commit [`56a9903`](https://github.com/apache/spark/commit/56a990316a947785b2502e3129c49ab00bef9b90).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440405708



##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -775,7 +802,12 @@ private[spark] class MapOutputTrackerMaster(
   override def stop(): Unit = {
     mapOutputRequests.offer(PoisonPill)
     threadpool.shutdown()
-    sendTracker(StopMapOutputTracker)
+    try {
+      sendTracker(StopMapOutputTracker)
+    } catch {
+      case e: Exception =>

Review comment:
       So the exception I want to catch is a `SparkException`, so I'll just narrow it to that.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660380089


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126073/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-640763276






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660315845


   **[Test build #126073 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126073/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639117770


   **[Test build #123542 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123542/testReport)** for PR 28708 at commit [`a904030`](https://github.com/apache/spark/commit/a904030d78ca9ad1e6da8de0359758cce8d58abb).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435446889



##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -479,6 +497,16 @@ private[spark] class MapOutputTrackerMaster(
     }
   }
 
+  def updateMapOutput(shuffleId: Int, mapId: Long, bmAddress: BlockManagerId): Unit = {
+    shuffleStatuses.get(shuffleId) match {
+      case Some(shuffleStatus) =>
+        shuffleStatus.updateMapOutput(mapId, bmAddress)
+        shuffleStatus.invalidateSerializedMapOutputStatusCache()

Review comment:
       Good catch, I'll drop it.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-641627813


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28333/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638577007






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439700057



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,265 @@
+/*
+ * 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
+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)
+
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = 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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                bm.migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              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 ${shuffleId},${mapId} 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[(Int, Long)]()
+
+  // Shuffles which are queued for migration
+  private[storage] val shufflesToMigrate =
+    new java.util.concurrent.ConcurrentLinkedQueue[(Int, Long)]()
+
+  @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)

Review comment:
       Cool, I'll add this in the code up above the loop where we launch the two different migrations :)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435522183



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} to ${peer}")

Review comment:
       Once the block was migrated to peer here, does it exist both on peer and in current block manager at the same time? If so, the request for the shuffle block will go to peer or current block manager before the current one is decommissioned?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643666176


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28604/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r447247802



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -40,6 +40,9 @@ sealed abstract class BlockId {
   def isRDD: Boolean = isInstanceOf[RDDBlockId]
   def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId]
   def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId]
+  def isInternalShuffle: Boolean = {

Review comment:
       Looking at it, not widely used I'll audit each use case and then decide.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642228474






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638537610


   **[Test build #123515 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123515/testReport)** for PR 28708 at commit [`ccb8827`](https://github.com/apache/spark/commit/ccb8827c61e3178c83ae505dbddd30b1a6a2d361).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660116404


   **[Test build #126054 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126054/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440378850



##########
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:
       no reason to, this is pretty close to a noop if we get a duplicated update message.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660114456


   test this please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435575094



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("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")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()

Review comment:
       No, if we get a class cast exception we want to bubble it up because there isn't anything we can do in that situation besides report it.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638684678


   > So I don't want to stop the executor directly once the block migration is done. Instead, I have a follow-up JIRA which I've started working on that shutdowns the executor once the block migration has completed and there are no running tasks. I think it's ok to (temporarily) leak threads since decommissioning (as triggered currently) is only in the situation where the executor will be exiting soon anyways.
   
   I understand your point but as we are testing the decommissioning these threads are leaking during testing and even aggregating by each test using this feature. 
   
   We are logging this leaks in this method https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/ThreadAudit.scala#L106 at the end of the suite. 
   
   I believe it is better to keep such log completely clean if the required code change is easy otherwise the next person felt to be encouraged to add just one more violation.
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644432278






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660387989


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126076/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645652389






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657937025


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/30410/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644468103


   I've updated the description @gatorsmile, let me know if there is any particular points you would like clarified.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637882815


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] HyukjinKwon removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
HyukjinKwon removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637902310


   cc @vanzin, @jiangxb1987, @Ngone51 FYI


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645674272






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439852420



##########
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")
+      .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " +
+        "an indexed shuffle resolver (like sort based shuffe)")

Review comment:
       It would be great if we have a defensive code somewhere in this PR for this requirement. For example, `assert()` in the code?
   > Requires an indexed shuffle resolver (like sort based shuffe)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660182731






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r468090216



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -148,6 +170,82 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * Write a provided shuffle block as a stream. Used for block migrations.
+   * ShuffleBlockBatchIds must contain the full range represented in the ShuffleIndexBlock.
+   * Requires the caller to delete any shuffle index blocks where the shuffle block fails to
+   * put.
+   */
+  override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager):
+      StreamCallbackWithID = {
+    val file = blockId match {
+      case ShuffleIndexBlockId(shuffleId, mapId, _) =>
+        getIndexFile(shuffleId, mapId)
+      case ShuffleDataBlockId(shuffleId, mapId, _) =>
+        getDataFile(shuffleId, mapId)
+      case _ =>
+        throw new Exception(s"Unexpected shuffle block transfer ${blockId} as " +
+          s"${blockId.getClass().getSimpleName()}")
+    }
+    val fileTmp = Utils.tempFileWith(file)
+    val channel = Channels.newChannel(
+      serializerManager.wrapStream(blockId,
+        new FileOutputStream(fileTmp)))
+
+    new StreamCallbackWithID {
+
+      override def getID: String = blockId.name
+
+      override def onData(streamId: String, buf: ByteBuffer): Unit = {
+        while (buf.hasRemaining) {
+          channel.write(buf)
+        }
+      }
+
+      override def onComplete(streamId: String): Unit = {
+        logTrace(s"Done receiving shuffle block $blockId, now storing on local disk.")
+        channel.close()
+        val diskSize = fileTmp.length()
+        this.synchronized {

Review comment:
       The lock used for synchronizing looks incorrect here - should be `IndexShuffleBlockResolver.this` ?
   (I missed reviewing this earlier, apologies)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-646209154


   I will merge early next week unless anyone has any outstanding issues.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440410753



##########
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")
+      .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " +
+        "an indexed shuffle resolver (like sort based shuffe)")

Review comment:
       Got it~




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644458813


   **[Test build #124075 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124075/testReport)** for PR 28708 at commit [`d9123f9`](https://github.com/apache/spark/commit/d9123f961d3738ebdb2899331cd6983476575504).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660396945






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659726448


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639095824


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643577401


   **[Test build #123958 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123958/testReport)** for PR 28708 at commit [`0ea927d`](https://github.com/apache/spark/commit/0ea927d9148089a3799b2e94a35589795967e819).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644422385


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28694/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637780168


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28066/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643451769


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28563/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643553994


   Yeah I think supporting multiple ways of reducing the number of fetch failures makes sense here. I think migration is certainly a "best-case" scenario and we can't count on in migrating everything in overcommit environments.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660182094


   **[Test build #126054 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126054/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440407328



##########
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")
+      .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " +
+        "an indexed shuffle resolver (like sort based shuffe)")

Review comment:
       Oh actually should change this wording, it can work a little more generally now. Any implementation which implements a given trait. I'm not sure about adding an assert here, we do log it when we first try and use it though.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643581963


   **[Test build #123964 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123964/testReport)** for PR 28708 at commit [`da1db47`](https://github.com/apache/spark/commit/da1db4740778b3f12df88e4c28aa0602ff15417e).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r434895820



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -650,6 +657,19 @@ private[spark] class BlockManager(
       blockId: BlockId,
       level: StorageLevel,
       classTag: ClassTag[_]): StreamCallbackWithID = {
+    // Delegate shuffle blocks here to resolver if supported
+    if (blockId.isShuffle || blockId.isInternalShuffle) {

Review comment:
       It is for the current implementation of the trait, but if someone else implemented this for something beyond IndexShuffle then it might not be.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638558200






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638458450


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28122/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-650481575


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/29178/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660394447


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30693/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660411447


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126080/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644969217






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659083173


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439851962



##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -775,7 +802,12 @@ private[spark] class MapOutputTrackerMaster(
   override def stop(): Unit = {
     mapOutputRequests.offer(PoisonPill)
     threadpool.shutdown()
-    sendTracker(StopMapOutputTracker)
+    try {
+      sendTracker(StopMapOutputTracker)
+    } catch {
+      case e: Exception =>

Review comment:
       Can we catch a more specific exception? This looks too general.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-640736229






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658000525






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657927290


   **[Test build #125799 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125799/testReport)** for PR 28708 at commit [`5a0cd2a`](https://github.com/apache/spark/commit/5a0cd2abd316aacc601b9e8fa6e1406b67c55fb7).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660380086






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639040196


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638540995






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644459506






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658000513


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639202750


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28172/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638496741


   So I don't want to stop the executor directly once the block migration is done. Instead, I have a follow-up JIRA which I've started working on that shutdowns the executor once the block migration has completed and there are no running tasks. I think it's ok to (temporarily) leak threads since decommissioning (as triggered currently) is only in the situation where the executor will be exiting soon anyways.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643582017






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456062326



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -168,7 +168,10 @@ private[spark] class NettyBlockTransferService(
     // Everything else is encoded using our binary protocol.
     val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag)))
 
-    val asStream = blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM)
+    // We always transfer shuffle blocks as a stream for simplicity with the receiving code since
+    // they are always written to disk. Otherwise we check the block size.
+    val asStream = (blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) ||

Review comment:
       I think it's easier to read as is.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658503902






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r455352852



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -44,9 +47,9 @@ import org.apache.spark.util.Utils
 // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData().
 private[spark] class IndexShuffleBlockResolver(
     conf: SparkConf,
-    _blockManager: BlockManager = null)
+    var _blockManager: BlockManager = null)

Review comment:
       this is a var for testing?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r445108498



##########
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:
       Maybe? I was thinking that info might be the right level for successfully migrated shuffle blocks, but if you think debug would be better I'm happy to drop it down a level.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-650477379


   **[Test build #124557 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124557/testReport)** for PR 28708 at commit [`ac096f4`](https://github.com/apache/spark/commit/ac096f46aa6e658704d7726efe7d66ece280b83e).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-645639404


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28796/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644407813


   **[Test build #124076 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124076/testReport)** for PR 28708 at commit [`6a77220`](https://github.com/apache/spark/commit/6a77220ae535aa449e74e679a0ad95a279398ae6).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660116404


   **[Test build #126054 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126054/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639071957


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/28166/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440910366



##########
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:
       I see.

##########
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:
       make sense.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657987750


   **[Test build #125806 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125806/testReport)** for PR 28708 at commit [`5a0cd2a`](https://github.com/apache/spark/commit/5a0cd2abd316aacc601b9e8fa6e1406b67c55fb7).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dbtsai commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660802517


   Thanks! This is a great milestone. 


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643451795






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639201319


   > > > So @attilapiros looking at the Jenkins console logs we aren't leaking any threads during testing (nor would I expect us to). But I'll add something to more aggressively stop the shuffle migration threads.
   > > 
   > > 
   > > It will come when the `BlockManager` will be tested in `BlockManagerSuite`:
   > > ```
   > >  ===== POSSIBLE THREAD LEAK IN SUITE o.a.s.storage.BlockManagerSuite, thread names: rpc-boss-3-1, migrate-shuffle-to-BlockManagerId(exec2, localhost, 50804, None), shuffle-boss-9-1  , shuffle-boss-6-1 =====
   > > ```
   > 
   > Gotcha was looking for the explicit decom test. I'll eagerly shutdown the migrate-shuffle-to threads then.
   
   I think the latest changes have fixed this (e.g. `grep "THREAD LEAK" consoleFull  |grep BlockManager` returns nothing). Worth noting  we do leak threads in ~283 tests so I'm not sure how important this is.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637896250






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435438297



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1725,6 +1725,16 @@ class SparkContext(config: SparkConf) extends Logging {
     }
   }
 
+
+  private[spark] def decommissionExecutors(executorIds: Seq[String]): Unit = {
+    schedulerBackend match {
+      case b: CoarseGrainedSchedulerBackend =>
+        executorIds.foreach(b.decommissionExecutor)
+      case _ =>
+        logWarning("Decommissioning executors is not supported by current scheduler.")

Review comment:
       Logging schedule name in logging message?

##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -479,6 +497,16 @@ private[spark] class MapOutputTrackerMaster(
     }
   }
 
+  def updateMapOutput(shuffleId: Int, mapId: Long, bmAddress: BlockManagerId): Unit = {
+    shuffleStatuses.get(shuffleId) match {
+      case Some(shuffleStatus) =>
+        shuffleStatus.updateMapOutput(mapId, bmAddress)
+        shuffleStatus.invalidateSerializedMapOutputStatusCache()

Review comment:
       `ShuffleStatus.updateMapOutput` calls `invalidateSerializedMapOutputStatusCache`, do we need calling it again here?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435447040



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -1725,6 +1725,16 @@ class SparkContext(config: SparkConf) extends Logging {
     }
   }
 
+
+  private[spark] def decommissionExecutors(executorIds: Seq[String]): Unit = {
+    schedulerBackend match {
+      case b: CoarseGrainedSchedulerBackend =>
+        executorIds.foreach(b.decommissionExecutor)
+      case _ =>
+        logWarning("Decommissioning executors is not supported by current scheduler.")

Review comment:
       Sure :)




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639007997


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123539/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639014498


   **[Test build #123541 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123541/testReport)** for PR 28708 at commit [`bce1613`](https://github.com/apache/spark/commit/bce1613a7be0afa9ce6b0404e824942e99d17396).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638593857






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643552975


   @holdenk, what are your thoughts on reducing the number of fetch failed exceptions (and thus potential job failures) due to decommissioning: 
   - This PR tries to offload the shuffle blocks to another peer but that is somewhat best effort. 
   - When the executor is eventually lost (detected much later by a heartbeat timeout), its shuffle files will be cleared. Until that happens the unmigrated blocks (for whatever reason) will result in fetch failures. 
   
   Block migration may not be possible in all cases since we may not have enough time to do the migration or the peer's disk might be full or it might soon after be decommissioned itself. This sort of "bulk decommissioning" can happen for example during a cluster scale down where many nodes are brought down in quick succession. 
   
   While block migration is great in that it avoids wasting work, we cannot guarantee it and I think we need a second line of defense by proactively calling `MapOutputTracker.removeOutputsOnExecutor` (and clearing `DAGScheduler.cacheLocs`) perhaps after some timeout. (I think FetchFailures may still happen if a reducer has already started.)
   
   It would be nice to prevent/reduce job failures caused by fetch failures: Particularly because we already know that the executor would be yanked soon, so we should be able to do better.
   
   Thanks.
   
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643679317






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643679317






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659713225


   **[Test build #126005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126005/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658167335


   **[Test build #125835 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125835/testReport)** for PR 28708 at commit [`546d953`](https://github.com/apache/spark/commit/546d953f0e2a8c7b5a9d9e7a476c9a0eca408417).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659083173






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644432278


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637753159


   **[Test build #123448 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123448/testReport)** for PR 28708 at commit [`4730a52`](https://github.com/apache/spark/commit/4730a5258d478b3b03a76dc994b65fe74c76997b).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660384743


   **[Test build #126080 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126080/testReport)** for PR 28708 at commit [`8494bdd`](https://github.com/apache/spark/commit/8494bdd94285c7cc5a41e151da920710be7f4671).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660353405


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30686/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660412762


   **[Test build #126078 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126078/testReport)** for PR 28708 at commit [`16b7376`](https://github.com/apache/spark/commit/16b7376f39cf8ba27fae898a5dd58ba6e64f38f9).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456062915



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -650,6 +662,23 @@ private[spark] class BlockManager(
       blockId: BlockId,
       level: StorageLevel,
       classTag: ClassTag[_]): StreamCallbackWithID = {
+
+    if (isDecommissioning()) {
+       throw new BlockSavedOnDecommissionedBlockManagerException(blockId)
+    }
+
+    if (blockId.isShuffle) {
+      logInfo(s"Putting shuffle block ${blockId}")

Review comment:
       Sure I can drop these two down to debug.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639661079


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28195/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440374116



##########
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:
       Good catch, I'll drop this.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637753159


   **[Test build #123448 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123448/testReport)** for PR 28708 at commit [`4730a52`](https://github.com/apache/spark/commit/4730a5258d478b3b03a76dc994b65fe74c76997b).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439183738



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
##########
@@ -0,0 +1,265 @@
+/*
+ * 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
+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)
+
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = 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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                bm.migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              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 ${shuffleId},${mapId} 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[(Int, Long)]()
+
+  // Shuffles which are queued for migration
+  private[storage] val shufflesToMigrate =
+    new java.util.concurrent.ConcurrentLinkedQueue[(Int, Long)]()
+
+  @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:
       This will spawn a thread per peer .. which is like per executor. I think we should bound this using a thread pool or something, otherwise this would scale with the cluster size. 




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637928830


   **[Test build #123452 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123452/testReport)** for PR 28708 at commit [`7fb4356`](https://github.com/apache/spark/commit/7fb4356890c319afee95a79fdb7cc41f127efabb).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait MigratableResolver `
     * `class RDDBlockSavedOnDecommissionedBlockManagerException(blockId: RDDBlockId)`


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638501022


   **[Test build #123511 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123511/testReport)** for PR 28708 at commit [`7dde80e`](https://github.com/apache/spark/commit/7dde80e5a698cdeee94f40e9c0fd7375c44f9420).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660363823


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439853227



##########
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)

Review comment:
       Does this mean an infinite loop when there is no remaining disk on the other executors?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643581618






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660827720


   Thank you all!


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439853479



##########
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) {

Review comment:
       Can we make this `20` as a configuration?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643669064






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643428071


   **[Test build #123941 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123941/testReport)** for PR 28708 at commit [`7f00072`](https://github.com/apache/spark/commit/7f00072a93053881cb6ee3f69f3be0560142337e).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r445073751



##########
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:
       Also to clarify: this race condition is OK here in this PR as well since we just re-run it until the executor is killed so no worries about missing files.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639054992


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/28164/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r454897777



##########
File path: core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.Semaphore
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+
+import org.scalatest.concurrent.Eventually
+
+import org.apache.spark._
+import org.apache.spark.internal.config
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
+import org.apache.spark.util.{ResetSystemProperties, ThreadUtils}
+
+class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext
+    with ResetSystemProperties with Eventually {
+
+  val numExecs = 3
+  val numParts = 3
+
+  test(s"verify that an already running task which is going to cache data succeeds " +
+    s"on a decommissioned executor") {
+    runDecomTest(true, false, true)
+  }
+
+  test(s"verify that shuffle blocks are migrated") {
+    runDecomTest(false, true, false)
+  }
+
+  test(s"verify that both migrations can work at the same time.") {
+    runDecomTest(true, true, false)
+  }
+
+  private def runDecomTest(persist: Boolean, shuffle: Boolean, migrateDuring: Boolean) = {
+
+    val master = s"local-cluster[${numExecs}, 1, 1024]"
+    val conf = new SparkConf().setAppName("test").setMaster(master)
+      .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_ENABLED, true)
+      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, persist)
+      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, shuffle)
+    // Just replicate blocks as fast as we can during testing, there isn't another
+    // workload we need to worry about.

Review comment:
       nit: indent




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439853174



##########
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 =>

Review comment:
       Could you use more specific `Exception` type?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643555962


   **[Test build #123958 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123958/testReport)** for PR 28708 at commit [`0ea927d`](https://github.com/apache/spark/commit/0ea927d9148089a3799b2e94a35589795967e819).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-659713225


   **[Test build #126005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126005/testReport)** for PR 28708 at commit [`9d210f5`](https://github.com/apache/spark/commit/9d210f53c1907a46a80b2887b47592f5c42b040a).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435508884



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1907,23 +2041,39 @@ private[spark] class BlockManager(
    */
   private class BlockManagerDecommissionManager(conf: SparkConf) {
     @volatile private var stopped = false
-    private val sleepInterval = conf.get(
-      config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL)
+    private val blockMigrationThread = new Thread {
+      val sleepInterval = conf.get(
+        config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL)
 
-    private val blockReplicationThread = new Thread {
       override def run(): Unit = {
         var failures = 0
         while (blockManagerDecommissioning
           && !stopped
           && !Thread.interrupted()
           && failures < 20) {
+          logInfo("Iterating on migrating from the block manager.")
           try {
-            logDebug("Attempting to replicate all cached RDD blocks")
-            decommissionRddCacheBlocks()
-            logInfo("Attempt to replicate all cached blocks done")
+            // If enabled we migrate shuffle blocks first as they are more expensive.
+            if (conf.get(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED)) {
+              logDebug(s"Attempting to replicate all shuffle blocks")
+              offloadShuffleBlocks()
+              logInfo(s"Done starting workers to migrate shuffle blocks")
+            }
+            if (conf.get(config.STORAGE_RDD_DECOMMISSION_ENABLED)) {
+              logDebug(s"Attempting to replicate all cached RDD blocks")
+              decommissionRddCacheBlocks()
+              logInfo(s"Attempt to replicate all cached blocks done")
+            }

Review comment:
       nit: no need for string interpolation s"".

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("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")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()
+    logInfo(s"My local shuffles are ${localShuffles.toList}")
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq

Review comment:
       `diff` should be better for readability.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1934,19 +2084,20 @@ private[spark] class BlockManager(
         }
       }
     }
-    blockReplicationThread.setDaemon(true)
-    blockReplicationThread.setName("block-replication-thread")
+    blockMigrationThread.setDaemon(true)
+    blockMigrationThread.setName("block-replication-thread")

Review comment:
       nit: block-migration-thread

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1934,19 +2084,20 @@ private[spark] class BlockManager(
         }
       }
     }
-    blockReplicationThread.setDaemon(true)
-    blockReplicationThread.setName("block-replication-thread")
+    blockMigrationThread.setDaemon(true)
+    blockMigrationThread.setName("block-replication-thread")

Review comment:
       And also applying on some logging info below.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("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")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()
+    logInfo(s"My local shuffles are ${localShuffles.toList}")
+    val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq
+    logInfo(s"My new shuffles to migrate ${newShufflesToMigrate.toList}")
+    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 = 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)

Review comment:
       Looks like now it migrates shuffle blocks to arbitrary peer. Will it be better to migrate shuffle blocks for the same shuffle id to same peer?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("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")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()
+    logInfo(s"My local shuffles are ${localShuffles.toList}")

Review comment:
       `logDebug`? Shuffle files could be long list and increase log size.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1822,108 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("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")
+          }
+      }
+    }
+  }
+
+  private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]()
+
+  /**
+   * 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.
+   */
+  def offloadShuffleBlocks(): Unit = {
+    // Update the queue of shuffles to be migrated
+    logInfo("Offloading shuffle blocks")
+    val localShuffles = migratableResolver.getStoredShuffles()

Review comment:
       Do we need to catch `ClassCastException` too here?




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-637884776


   **[Test build #123452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123452/testReport)** for PR 28708 at commit [`7fb4356`](https://github.com/apache/spark/commit/7fb4356890c319afee95a79fdb7cc41f127efabb).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r439384531



##########
File path: core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.shuffle
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.client.StreamCallbackWithID
+import org.apache.spark.serializer.SerializerManager
+import org.apache.spark.storage.BlockId
+
+/**
+ * :: Experimental ::
+ * An experimental trait to allow Spark to migrate shuffle blocks.
+ */
+@Experimental
+trait MigratableResolver {
+  /**
+   * Get the shuffle ids that are stored locally. Used for block migrations.

Review comment:
       Sure, sounds reasonable.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638501022


   **[Test build #123511 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123511/testReport)** for PR 28708 at commit [`7dde80e`](https://github.com/apache/spark/commit/7dde80e5a698cdeee94f40e9c0fd7375c44f9420).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456892174



##########
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

Review comment:
       I think this is fine as I have seen this pattern used in the code somewhere else too, ie.:
   https://github.com/apache/spark/blob/026b0b926dfd40038f2cee932f38b917eb25b77e/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L520
   
   I believe this is for emphasizing here is something to be set only once with a correct default (by code as opposed to a config param). So reading the code it raises attention that here an important choice was made. 

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -148,6 +170,82 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * Write a provided shuffle block as a stream. Used for block migrations.
+   * ShuffleBlockBatchIds must contain the full range represented in the ShuffleIndexBlock.
+   * Requires the caller to delete any shuffle index blocks where the shuffle block fails to
+   * put.
+   */
+  override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager):
+      StreamCallbackWithID = {
+    val file = blockId match {
+      case ShuffleIndexBlockId(shuffleId, mapId, _) =>
+        getIndexFile(shuffleId, mapId)
+      case ShuffleDataBlockId(shuffleId, mapId, _) =>
+        getDataFile(shuffleId, mapId)
+      case _ =>
+        throw new Exception(s"Unexpected shuffle block transfer ${blockId} as " +
+          s"${blockId.getClass().getSimpleName()}")
+    }
+    val fileTmp = Utils.tempFileWith(file)
+    val channel = Channels.newChannel(
+      serializerManager.wrapStream(blockId,
+        new FileOutputStream(fileTmp)))
+
+    new StreamCallbackWithID {
+
+      override def getID: String = blockId.name
+
+      override def onData(streamId: String, buf: ByteBuffer): Unit = {
+        while (buf.hasRemaining) {
+          channel.write(buf)
+        }
+      }
+
+      override def onComplete(streamId: String): Unit = {
+        logTrace(s"Done receiving shuffle block $blockId, now storing on local disk.")
+        channel.close()
+        val diskSize = fileTmp.length()
+        this.synchronized {

Review comment:
       I think this lock might not be needed as in `writeIndexFileAndCommit` the lock is used to have the `checkIndexAndDataFile` and the file rename in one atomic operation. 
   
   But in a follow-up PR we can investigate this more.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660364460


   sounds good, I'll work on resolving the test issues. @jiangxb1987 if you want to make the follow up issues under the decommissioning umbrella issue it'll make tracking it easier :)


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-644955838






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-638523253






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r435375916



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1790,6 +1817,100 @@ private[spark] class BlockManager(
     }
   }
 
+  private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable {
+    @volatile var running = true
+    override def run(): Unit = {
+      var migrating: Option[(Int, Long)] = None
+      val storageLevel = StorageLevel(
+        useDisk = true,
+        useMemory = false,
+        useOffHeap = false,
+        deserialized = false,
+        replication = 1)
+      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) {
+          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((shuffleId, mapId)) =>
+              logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}")
+              val blocks =
+                migratableResolver.getMigrationBlocks(shuffleId, mapId)
+              logInfo(s"Got migration sub-blocks ${blocks}")
+              blocks.foreach { case (blockId, buffer) =>
+                logInfo(s"Migrating sub-block ${blockId}")
+                blockTransferService.uploadBlockSync(
+                  peer.host,
+                  peer.port,
+                  peer.executorId,
+                  blockId,
+                  buffer,
+                  storageLevel,
+                  null)// class tag, we don't need for shuffle
+                logInfo(s"Migrated sub block ${blockId}")
+              }
+              logInfo(s"Migrated ${shuffleId},${mapId} 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("Error ${e} during migration, adding ${shuffleMap} back to migration queue")
+              shufflesToMigrate.add(shuffleMap)
+            case None =>
+              logError("Error ${e} while waiting for block to migrate")

Review comment:
       ```suggestion
                 logError(s"Error ${e} while waiting for block to migrate")
   ```




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639769322






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r440370509



##########
File path: core/src/main/scala/org/apache/spark/MapOutputTracker.scala
##########
@@ -775,7 +802,12 @@ private[spark] class MapOutputTrackerMaster(
   override def stop(): Unit = {
     mapOutputRequests.offer(PoisonPill)
     threadpool.shutdown()
-    sendTracker(StopMapOutputTracker)
+    try {
+      sendTracker(StopMapOutputTracker)
+    } catch {
+      case e: Exception =>

Review comment:
       Good catch, I'll narrow this down.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658503902


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-657927290


   **[Test build #125799 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125799/testReport)** for PR 28708 at commit [`5a0cd2a`](https://github.com/apache/spark/commit/5a0cd2abd316aacc601b9e8fa6e1406b67c55fb7).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658910619






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-639628456


   **[Test build #123572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123572/testReport)** for PR 28708 at commit [`2ca6c08`](https://github.com/apache/spark/commit/2ca6c08e00b9378365a7f4bd8fccec9f85960b58).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-643577621






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28708:
URL: https://github.com/apache/spark/pull/28708#discussion_r456115333



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -55,6 +58,24 @@ private[spark] class IndexShuffleBlockResolver(
 
   def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None)
 
+  /**
+   * Get the shuffle files that are stored locally. Used for block migrations.
+   */
+  override def getStoredShuffles(): Set[ShuffleBlockInfo] = {
+    // Matches ShuffleIndexBlockId name
+    val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r
+    val rootDirs = blockManager.diskBlockManager.localDirs
+    // ExecutorDiskUtil puts things inside one level hashed sub directories
+    val searchDirs = rootDirs.flatMap(_.listFiles()).filter(_.isDirectory()) ++ rootDirs

Review comment:
       Since your other comment, I managed to drop this part so we can avoid this code and the regex :)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-641532764


   **[Test build #123704 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123704/testReport)** for PR 28708 at commit [`8f7d4eb`](https://github.com/apache/spark/commit/8f7d4ebc601992d84f44c9040d520fcb8015e389).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28708: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-642228474


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-660353199


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/30689/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28708:
URL: https://github.com/apache/spark/pull/28708#issuecomment-658290185


   **[Test build #125835 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125835/testReport)** for PR 28708 at commit [`546d953`](https://github.com/apache/spark/commit/546d953f0e2a8c7b5a9d9e7a476c9a0eca408417).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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