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

2020-06-04 Thread GitBox


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



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-04 Thread GitBox


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?





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

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



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



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

2020-06-04 Thread GitBox


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 = 

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

2020-06-04 Thread GitBox


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



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