From 094d5844226232de3518cada540f71e663ec1438 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 2 Jun 2020 12:09:01 -0700 Subject: [PATCH 01/47] Add an option to migrate shuffle blocks as well as the current cache blocks during decommissioning --- .../org/apache/spark/MapOutputTracker.scala | 32 ++- .../scala/org/apache/spark/SparkContext.scala | 12 +- .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../spark/internal/config/package.scala | 15 ++ .../apache/spark/scheduler/MapStatus.scala | 12 +- .../cluster/StandaloneSchedulerBackend.scala | 2 +- .../shuffle/IndexShuffleBlockResolver.scala | 106 ++++++++- .../spark/shuffle/MigratableResolver.scala | 47 ++++ .../org/apache/spark/storage/BlockId.scala | 3 + .../apache/spark/storage/BlockManager.scala | 168 ++++++++++++-- .../spark/storage/BlockManagerMaster.scala | 1 - .../storage/BlockManagerMasterEndpoint.scala | 28 ++- ...nDecommissionedBlockManagerException.scala | 21 ++ .../scheduler/WorkerDecommissionSuite.scala | 2 +- .../sort/IndexShuffleBlockResolverSuite.scala | 3 + .../BlockManagerDecommissionSuite.scala | 207 +++++++++++++++--- .../BlockManagerReplicationSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 2 +- .../integrationtest/DecommissionSuite.scala | 13 +- .../k8s/integrationtest/KubernetesSuite.scala | 22 +- .../tests/decommissioning.py | 25 ++- .../streaming/ReceivedBlockHandlerSuite.scala | 2 +- 22 files changed, 655 insertions(+), 73 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ec8621bc55cf3..43370ee31abde 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -49,7 +49,7 @@ import org.apache.spark.util._ * * All public methods of this class are thread-safe. */ -private class ShuffleStatus(numPartitions: Int) { +private class ShuffleStatus(numPartitions: Int) extends Logging { private val (readLock, writeLock) = { val lock = new ReentrantReadWriteLock() @@ -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("Updating map output for ${mapId} to ${bmAddress}") + mapStatus.updateLocation(bmAddress) + invalidateSerializedMapOutputStatusCache() + case None => + logError("Asked to update map output ${mapId} for untracked map status.") + } + } + /** * Remove the map output which was served by the specified block manager. * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock { + logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}") if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { _numAvailableOutputs -= 1 mapStatuses(mapIndex) = null @@ -139,6 +155,7 @@ private class ShuffleStatus(numPartitions: Int) { * outputs which are served by an external shuffle server (if one exists). */ def removeOutputsOnHost(host: String): Unit = withWriteLock { + logDebug(s"Removing outputs for host ${host}") removeOutputsByFilter(x => x.host == host) } @@ -148,6 +165,7 @@ private class ShuffleStatus(numPartitions: Int) { * still registered with that execId. */ def removeOutputsOnExecutor(execId: String): Unit = withWriteLock { + logDebug(s"Removing outputs for execId ${execId}") removeOutputsByFilter(x => x.executorId == execId) } @@ -265,7 +283,7 @@ private[spark] class MapOutputTrackerMasterEndpoint( override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = context.senderAddress.hostPort - logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) + logInfo(s"Asked to send map output locations for shuffle ${shuffleId} to ${hostPort}") tracker.post(new GetMapOutputMessage(shuffleId, context)) case StopMapOutputTracker => @@ -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() + case None => + logError(s"Asked to update map output for unknown shuffle ${shuffleId}") + } + } + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus): Unit = { shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 38d7319b1f0ef..c36bbe9b0810f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -57,7 +57,7 @@ import org.apache.spark.resource._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.shuffle.ShuffleDataIOUtils import org.apache.spark.shuffle.api.ShuffleDriverComponents @@ -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.") + } + } + /** The version of Spark on which this application is running. */ def version: String = SPARK_VERSION diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8ba1739831803..d543359f4dedf 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -367,7 +367,8 @@ object SparkEnv extends Logging { externalShuffleClient } else { None - }, blockManagerInfo)), + }, blockManagerInfo, + mapOutputTracker.asInstanceOf[MapOutputTrackerMaster])), registerOrLookupEndpoint( BlockManagerMaster.DRIVER_HEARTBEAT_ENDPOINT_NAME, new BlockManagerMasterHeartbeatEndpoint(rpcEnv, isLocal, blockManagerInfo)), diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ee437c696b47e..009f38aa0574a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/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)") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_RDD_DECOMMISSION_ENABLED = + ConfigBuilder("spark.storage.decommission.rdd_blocks") + .doc("Whether to transfer RDD blocks during block manager decommissioning.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + private[spark] val STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK = ConfigBuilder("spark.storage.decommission.maxReplicationFailuresPerBlock") .internal() diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 7f8893ff3b9d8..9dee1f779bcb5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -33,9 +33,11 @@ import org.apache.spark.util.Utils * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. */ private[spark] sealed trait MapStatus { - /** Location where this task was run. */ + /** Location where this task output is. */ def location: BlockManagerId + def updateLocation(bm: BlockManagerId): Unit + /** * Estimated size for the reduce block, in bytes. * @@ -126,6 +128,10 @@ private[spark] class CompressedMapStatus( override def location: BlockManagerId = loc + override def updateLocation(bm: BlockManagerId): Unit = { + loc = bm + } + override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } @@ -178,6 +184,10 @@ private[spark] class HighlyCompressedMapStatus private ( override def location: BlockManagerId = loc + override def updateLocation(bm: BlockManagerId): Unit = { + loc = bm + } + override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 42c46464d79e1..ff41957779c83 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -44,7 +44,7 @@ private[spark] class StandaloneSchedulerBackend( with StandaloneAppClientListener with Logging { - private var client: StandaloneAppClient = null + private[spark] var client: StandaloneAppClient = null private val stopping = new AtomicBoolean(false) private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sc.conf diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index af2c82e771970..79323be45845c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import java.io._ +import java.nio.ByteBuffer import java.nio.channels.Channels import java.nio.file.Files @@ -25,8 +26,10 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExecutorDiskUtils +import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -46,7 +49,7 @@ private[spark] class IndexShuffleBlockResolver( conf: SparkConf, _blockManager: BlockManager = null) extends ShuffleBlockResolver - with Logging { + with Logging with MigratableResolver { private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager) @@ -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[(Int, Long)] = { + // 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 => (matched.group(1).toInt, matched.group(2).toLong) + } + }.toSet + } + + /** * Get the shuffle data file. * @@ -148,6 +170,86 @@ 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 block $blockId, now putting into local shuffle service") + channel.close() + val diskSize = fileTmp.length() + this.synchronized { + if (file.exists()) { + file.delete() + } + if (!fileTmp.renameTo(file)) { + throw new IOException(s"fail to rename file ${fileTmp} to ${file}") + } + } + blockManager.reportBlockStatus(blockId, BlockStatus( + StorageLevel( + useDisk = true, + useMemory = false, + useOffHeap = false, + deserialized = false, + replication = 0) + , 0, diskSize)) + } + + override def onFailure(streamId: String, cause: Throwable): Unit = { + // the framework handles the connection itself, we just need to do local cleanup + channel.close() + fileTmp.delete() + } + } + } + + /** + * Get the index & data block for migration. + */ + def getMigrationBlocks(shuffleId: Int, mapId: Long): List[(BlockId, ManagedBuffer)] = { + // Load the index block + val indexFile = getIndexFile(shuffleId, mapId) + val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val indexFileSize = indexFile.length() + val indexBlockData = new FileSegmentManagedBuffer(transportConf, indexFile, 0, indexFileSize) + + // Load the data block + val dataFile = getDataFile(shuffleId, mapId) + val dataBlockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + val dataBlockData = new FileSegmentManagedBuffer(transportConf, dataFile, 0, dataFile.length()) + List((indexBlockId, indexBlockData), (dataBlockId, dataBlockData)) + } + + /** * Write an index file with the offsets of each block, plus a final offset at the end for the * end of the output file. This will be used by getBlockData to figure out where each block @@ -169,7 +271,7 @@ private[spark] class IndexShuffleBlockResolver( val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. - synchronized { + this.synchronized { val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) if (existingLengths != null) { // Another attempt for the same task has already written our map outputs successfully, diff --git a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala new file mode 100644 index 0000000000000..768d4f8db4364 --- /dev/null +++ b/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. + */ + def getStoredShuffles(): Set[(Int, Long)] + + /** + * Write a provided shuffle block as a stream. Used for block migrations. + */ + def putShuffleBlockAsStream(blockId: BlockId, serializerManager: SerializerManager): + StreamCallbackWithID + + /** + * Get the blocks for migration for a particular shuffle and map. + */ + def getMigrationBlocks(shuffleId: Int, mapId: Long): List[(BlockId, ManagedBuffer)] +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 68ed3aa5b062f..398db1f4875ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/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 = { + isInstanceOf[ShuffleDataBlockId] || isInstanceOf[ShuffleIndexBlockId] + } override def toString: String = name } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e0478ad09601d..a91b27a0ef295 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -24,6 +24,7 @@ import java.nio.channels.Channels import java.util.Collections import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.HashMap import scala.concurrent.{ExecutionContext, Future} @@ -53,6 +54,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.{MigratableResolver, ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock import org.apache.spark.storage.memory._ @@ -254,6 +256,10 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + private lazy val migratableResolver: MigratableResolver = { + shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] + } + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -617,6 +623,7 @@ private[spark] class BlockManager( */ override def getLocalBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { + logInfo(s"Getting local shuffle block ${blockId}") shuffleManager.shuffleBlockResolver.getBlockData(blockId) } else { getLocalBytes(blockId) match { @@ -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) { + logInfo(s"Putting shuffle block ${blockId}") + try { + return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) + } catch { + case e: ClassCastException => throw new Exception( + s"Unexpected shuffle block ${blockId} with unsupported shuffle " + + s"resolver ${shuffleManager.shuffleBlockResolver}") + } + } + logInfo(s"Putting regular block ${blockId}") + // All other blocks val (_, tmpFile) = diskBlockManager.createTempLocalBlock() val channel = new CountingWritableChannel( Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile)))) @@ -720,7 +740,7 @@ private[spark] class BlockManager( * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - private def reportBlockStatus( + private[spark] def reportBlockStatus( blockId: BlockId, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { @@ -1285,6 +1305,9 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + if (blockManagerDecommissioning && blockId.isRDD) { + throw new RDDBlockSavedOnDecommissionedBlockManagerException(blockId.asRDDId.get) + } val putBlockInfo = { val newInfo = new BlockInfo(level, classTag, tellMaster) @@ -1781,7 +1804,7 @@ private[spark] class BlockManager( def decommissionBlockManager(): Unit = { if (!blockManagerDecommissioning) { - logInfo("Starting block manager decommissioning process") + logInfo("Starting block manager decommissioning process...") blockManagerDecommissioning = true decommissionManager = Some(new BlockManagerDecommissionManager(conf)) decommissionManager.foreach(_.start()) @@ -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) + (peer, runnable) + } + // A peer may have entered a decommissioning state, don't transfer any new blocks + deadPeers.foreach{peer => + migrationPeers.get(peer).foreach(_.running = false) + } + } + /** * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers * Visible for testing @@ -1798,7 +1922,7 @@ private[spark] class BlockManager( val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " + + logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + "for block manager decommissioning") } else { logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") @@ -1907,23 +2031,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") + } + 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 _: InterruptedException => + case e: InterruptedException => logInfo("Interrupted during migration, will not refresh migrations.") stopped = true case NonFatal(e) => @@ -1934,19 +2074,19 @@ private[spark] class BlockManager( } } } - blockReplicationThread.setDaemon(true) - blockReplicationThread.setName("block-replication-thread") + blockMigrationThread.setDaemon(true) + blockMigrationThread.setName("block-replication-thread") def start(): Unit = { logInfo("Starting block replication thread") - blockReplicationThread.start() + blockMigrationThread.start() } def stop(): Unit = { if (!stopped) { stopped = true logInfo("Stopping block replication thread") - blockReplicationThread.interrupt() + blockMigrationThread.interrupt() } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 3cfa5d2a25818..ff0b099e44b51 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.storage import scala.collection.generic.CanBuildFrom diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index d936420a99276..8c9dd42995064 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -28,7 +28,7 @@ import scala.util.Random import com.google.common.cache.CacheBuilder -import org.apache.spark.SparkConf +import org.apache.spark.{MapOutputTrackerMaster, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.ExternalBlockStoreClient @@ -48,7 +48,8 @@ class BlockManagerMasterEndpoint( conf: SparkConf, listenerBus: LiveListenerBus, externalBlockStoreClient: Option[ExternalBlockStoreClient], - blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) + blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo], + mapOutputTracker: MapOutputTrackerMaster) extends IsolatedRpcEndpoint with Logging { // Mapping from executor id to the block manager's local disk directories. @@ -157,7 +158,8 @@ class BlockManagerMasterEndpoint( context.reply(true) case DecommissionBlockManagers(executorIds) => - decommissionBlockManagers(executorIds.flatMap(blockManagerIdByExecutor.get)) + val bmIds = executorIds.flatMap(blockManagerIdByExecutor.get) + decommissionBlockManagers(bmIds) context.reply(true) case GetReplicateInfoForRDDBlocks(blockManagerId) => @@ -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 } // Remove a block from the slaves that have it. This can only be used to remove @@ -489,6 +492,23 @@ 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}") + return true + case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => + logInfo("Received shuffle data block update for ${shuffleId} ${mapId}, performing update") + mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) + return true + case _ => + logDebug(s"Unexpected shuffle block type ${blockId}") + return false + } + } if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { diff --git a/core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala b/core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala new file mode 100644 index 0000000000000..e6cef4dcc5e38 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class RDDBlockSavedOnDecommissionedBlockManagerException(blockId: RDDBlockId) + extends Exception(s"RDD Block $blockId cannot be saved on decommissioned executor") diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index 148d20ee659a2..cd3ab4db77f85 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -58,7 +58,7 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { }) TestUtils.waitUntilExecutorsUp(sc = sc, numExecutors = 2, - timeout = 10000) // 10s + timeout = 30000) // 30s val sleepyRdd = input.mapPartitions{ x => Thread.sleep(5000) // 5s x diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 27bb06b4e0636..b4092be466e27 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -48,6 +48,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(diskBlockManager.getFile(any[BlockId])).thenAnswer( (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) + when(diskBlockManager.localDirs).thenReturn(Array(tempDir)) } override def afterEach(): Unit = { @@ -73,6 +74,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) + val storedShuffles = resolver.getStoredShuffles() val indexFile = new File(tempDir.getAbsolutePath, idxName) val dataFile = resolver.getDataFile(shuffleId, mapId) @@ -81,6 +83,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(dataFile.exists()) assert(dataFile.length() === 30) assert(!dataTmp.exists()) + assert(storedShuffles === Set((1, 2))) val lengths2 = new Array[Long](3) val dataTmp2 = File.createTempFile("shuffle", null, tempDir) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala index 7456ca7f02a2e..f70aecd8793bb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala @@ -22,85 +22,226 @@ import java.util.concurrent.Semaphore import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, Success} +import org.scalatest.concurrent.Eventually + +import org.apache.spark._ import org.apache.spark.internal.config -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext - with ResetSystemProperties { + 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) + } - override def beforeEach(): Unit = { - val conf = new SparkConf().setAppName("test") + test(s"verify that shuffle blocks are migrated with force to disk") { + runDecomTest(false, true, false, remoteBlockSize = "1") + } + + 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, + remoteBlockSize: String = "100000") = { + + 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_RDD_DECOMMISSION_ENABLED, persist) + .set(config.STORAGE_SHUFFLE_DECOMMISSION_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("local-cluster[2, 1, 1024]", "test", conf) - } + // Allow force fetching to local disk + conf.set("spark.network.maxRemoteBlockSizeFetchToMem", remoteBlockSize) + + sc = new SparkContext(master, "test", conf) - test(s"verify that an already running task which is going to cache data succeeds " + - s"on a decommissioned executor") { // Create input RDD with 10 partitions - val input = sc.parallelize(1 to 10, 10) + val input = sc.parallelize(1 to numParts, numParts) val accum = sc.longAccumulator("mapperRunAccumulator") // Do a count to wait for the executors to be registered. 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 sleepyRdd = input.mapPartitions { x => - Thread.sleep(500) + val baseRdd = input.mapPartitions { x => + if (migrateDuring) { + Thread.sleep(500) + } accum.add(1) - x + x.map(y => (y, y)) + } + val testRdd = shuffle match { + case true => baseRdd.reduceByKey(_ + _) + case false => baseRdd } - // Listen for the job - val sem = new Semaphore(0) + // 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 = { - sem.release() + 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 - sleepyRdd.persist() + if (persist) { + testRdd.persist() + } - // Start the computation of RDD - this step will also cache the RDD - val asyncCount = sleepyRdd.countAsync() + // Wait for the first executor to start + TestUtils.waitUntilExecutorsUp(sc = sc, + numExecutors = 1, + timeout = 20000) // 20s - // Wait for the job to have started - sem.acquire(1) + // Start the computation of RDD - this step will also cache the RDD + val asyncCount = testRdd.countAsync() + + // Wait for all of the executors to start + TestUtils.waitUntilExecutorsUp(sc = sc, + // We need to make sure there is the original plus one exec to migrate too, we don't need + // the full set. + numExecutors = 2, + timeout = 30000) // 30s + + // 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) + } - // Give Spark a tiny bit to start the tasks after the listener says hello - Thread.sleep(100) // Decommission one of the executor val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] val execs = sched.getExecutorIds() - assert(execs.size == 2, s"Expected 2 executors but found ${execs.size}") + 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, 6.seconds) - assert(asyncCountResult === 10) - // All 10 tasks finished, so accum should have been increased 10 times - assert(accum.value === 10) + 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) // All tasks should be successful, nothing should have failed sc.listenerBus.waitUntilEmpty() - assert(taskEndEvents.size === 10) // 10 mappers - assert(taskEndEvents.map(_.reason).toSet === Set(Success)) + 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})") + } - // Since the RDD is cached, so further usage of same RDD should use the + // Wait for our respective blocks to have migrated + eventually(timeout(15.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}") + assert(numIndexLocs >= 1, s"Expect shuffle index block updates in ${blocksUpdated}") + } + } + + // Since the RDD is cached or shuffled so further usage of same RDD should use the + // cached data. Original RDD partitions should not be recomputed i.e. accum + // should have same value like before + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + + val storageStatus = sc.env.blockManager.master.getStorageStatus + val execIdToBlocksMapping = storageStatus.map( + status => (status.blockManagerId.executorId, status.blocks)).toMap + // No cached blocks should be present on executor which was decommissioned + assert(execIdToBlocksMapping(execToDecommission).keys.filter(_.isRDD).toSeq === Seq(), + "Cache blocks should be migrated") + if (persist) { + // There should still be all the RDD blocks cached + assert(execIdToBlocksMapping.values.flatMap(_.keys).count(_.isRDD) === numParts) + } + + // Make the executor we decommissioned exit + sched.client.killExecutors(List(execToDecommission)) + + // Wait for the executor to be removed + executorRemovedSem.acquire(1) + + // Since the RDD is cached or shuffled so further usage of same RDD should use the // cached data. Original RDD partitions should not be recomputed i.e. accum // should have same value like before - assert(sleepyRdd.count() === 10) - assert(accum.value === 10) + assert(testRdd.count() === numParts) + assert(accum.value === numParts) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 660bfcfc48267..d18d84dfaa9e5 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -103,7 +103,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) allStores.clear() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index bfef8f1ab29d8..4143129810aa3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -149,7 +149,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE liveListenerBus = spy(new LiveListenerBus(conf)) master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - liveListenerBus, None, blockManagerInfo)), + liveListenerBus, None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index becf9415c7506..e71d9ea127d25 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -28,18 +28,29 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => .set(Worker.WORKER_DECOMMISSION_ENABLED.key, "true") .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.container.image", pyImage) + .set("spark.storage.decommission.enabled", "true") + .set("spark.storage.decommission.shuffle_blocks", "true") + .set("spark.storage.decommission.shuffle_blocks", "true") + // Ensure we have somewhere to migrate our data too + .set("spark.executor.instances", "3") + // The default of 30 seconds is fine, but for testing we just want to get this done fast. + .set("spark.storage.decommission.replicationReattemptInterval", "1s") + .set("spark.storage.decommission.rdd_blocks", "true") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_DECOMISSIONING, mainClass = "", expectedLogOnCompletion = Seq( "Finished waiting, stopping Spark", - "decommissioning executor"), + "decommissioning executor", + "Final accumulator value is: 100"), appArgs = Array.empty[String], driverPodChecker = doBasicDriverPyPodCheck, executorPodChecker = doBasicExecutorPyPodCheck, appLocator = appLocator, isJVM = false, + pyFiles = None, + executorPatience = None, decommissioningTest = true) } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 65a2f1ff79697..098fe7d2562bc 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -325,7 +325,7 @@ class KubernetesSuite extends SparkFunSuite val result = checkPodReady(namespace, name) result shouldBe (true) } - // Look for the string that indicates we're good to clean up + // Look for the string that indicates we're good to trigger decom // on the driver logDebug("Waiting for first collect...") Eventually.eventually(TIMEOUT, INTERVAL) { @@ -333,13 +333,29 @@ class KubernetesSuite extends SparkFunSuite .pods() .withName(driverPodName) .getLog - .contains("Waiting to give nodes time to finish."), + .contains("Waiting to give nodes time to finish migration, decom exec 1."), "Decommission test did not complete first collect.") } // Delete the pod to simulate cluster scale down/migration. - val pod = kubernetesTestComponents.kubernetesClient.pods().withName(name) + // This will allow the pod to remain up for the grace period + val pod = kubernetesTestComponents.kubernetesClient.pods() + .withName(name) pod.delete() logDebug(s"Triggered pod decom/delete: $name deleted") + // Look for the string that indicates we should force kill the first + // Executor. This simulates the pod being fully lost. + logDebug("Waiting for second collect...") + Eventually.eventually(TIMEOUT, INTERVAL) { + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPodName) + .getLog + .contains("Waiting some more, please kill exec 1."), + "Decommission test did not complete second collect.") + } + logDebug("Force deleting") + val podNoGrace = pod.withGracePeriod(0) + podNoGrace.delete() } case Action.DELETED | Action.ERROR => execPods.remove(name) diff --git a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py index f68f24d49763d..84dc514b8988b 100644 --- a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py +++ b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py @@ -31,14 +31,29 @@ .appName("PyMemoryTest") \ .getOrCreate() sc = spark._sc - rdd = sc.parallelize(range(10)) + acc = sc.accumulator(0) + + def addToAcc(x): + acc.add(1) + return x + + initialRdd = sc.parallelize(range(100), 5) + accRdd = initialRdd.map(addToAcc) + # Trigger a shuffle so there are shuffle blocks to migrate + rdd = accRdd.map(lambda x: (x, x)).groupByKey() rdd.collect() - print("Waiting to give nodes time to finish.") + print("1st accumulator value is: " + str(acc.value)) + print("Waiting to give nodes time to finish migration, decom exec 1.") + print("...") time.sleep(5) + rdd.count() + print("Waiting some more, please kill exec 1.") + print("...") + time.sleep(5) + print("Executor node should be deleted now") + rdd.count() rdd.collect() - print("Waiting some more....") - time.sleep(10) - rdd.collect() + print("Final accumulator value is: " + str(acc.value)) print("Finished waiting, stopping Spark.") spark.stop() print("Done, exiting Python") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 0976494b6d094..558e2c99e0442 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -91,7 +91,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + new LiveListenerBus(conf), None, blockManagerInfo, mapOutputTracker)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true) From cd0781f2dc0651ba7ae0c324bd2da79bcc7f0499 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 3 Jun 2020 15:39:19 -0700 Subject: [PATCH 02/47] Update core/src/main/scala/org/apache/spark/storage/BlockManager.scala CR feedback "Nit: I think the comment is not needed as your code is self-explanatory here:" Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a91b27a0ef295..67e96c6f494d1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -657,7 +657,6 @@ private[spark] class BlockManager( blockId: BlockId, level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { - // Delegate shuffle blocks here to resolver if supported if (blockId.isShuffle || blockId.isInternalShuffle) { logInfo(s"Putting shuffle block ${blockId}") try { From 8e0304f7ed6019ef69fd7fda2f02cd9774edb8a2 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 3 Jun 2020 15:45:38 -0700 Subject: [PATCH 03/47] Update core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala If we have a failure during block migration, log the exception. Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- .../org/apache/spark/shuffle/IndexShuffleBlockResolver.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 79323be45845c..36ee3892017d3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -226,6 +226,7 @@ private[spark] class IndexShuffleBlockResolver( override def onFailure(streamId: String, cause: Throwable): Unit = { // the framework handles the connection itself, we just need to do local cleanup + logWarning(s"Error while uploading $blockId", cause) channel.close() fileTmp.delete() } From ac31c9085757e0e8076156d040e1d90de8ee73be Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 3 Jun 2020 15:48:13 -0700 Subject: [PATCH 04/47] First pass at the feedback from @attilapiros, mostly minor re-arrangement --- .../spark/shuffle/IndexShuffleBlockResolver.scala | 6 +++--- .../org/apache/spark/storage/BlockManager.scala | 12 +++++------- .../storage/BlockManagerDecommissionSuite.scala | 8 ++++---- .../deploy/k8s/integrationtest/KubernetesSuite.scala | 3 +-- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 36ee3892017d3..7886e02b8b10d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -69,7 +69,7 @@ private[spark] class IndexShuffleBlockResolver( val searchDirs = rootDirs.flatMap(_.listFiles()).filter(_.isDirectory()) ++ rootDirs val filenames = searchDirs.flatMap(_.list()) logDebug(s"Got block files ${filenames.toList}") - filenames.flatMap{ fname => + filenames.flatMap { fname => pattern.findAllIn(fname).matchData.map { matched => (matched.group(1).toInt, matched.group(2).toLong) } @@ -180,7 +180,7 @@ private[spark] class IndexShuffleBlockResolver( StreamCallbackWithID = { val file = blockId match { case ShuffleIndexBlockId(shuffleId, mapId, _) => - getIndexFile(shuffleId, mapId) + getIndexFile(shuffleId, mapId) case ShuffleDataBlockId(shuffleId, mapId, _) => getDataFile(shuffleId, mapId) case _ => @@ -203,7 +203,7 @@ private[spark] class IndexShuffleBlockResolver( } override def onComplete(streamId: String): Unit = { - logTrace(s"Done receiving block $blockId, now putting into local shuffle service") + logTrace(s"Done receiving shuffle block $blockId, now storing on local disk.") channel.close() val diskSize = fileTmp.length() this.synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 67e96c6f494d1..e59195343712f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -260,6 +260,11 @@ private[spark] class BlockManager( shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] } + // 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)]() + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -1812,13 +1817,6 @@ 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 = { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala index f70aecd8793bb..864a2a490b2ba 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala @@ -187,10 +187,10 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext eventually(timeout(15.seconds), interval(10.milliseconds)) { if (persist) { // One of our blocks should have moved. - val rddUpdates = blocksUpdated.filter{update => + val rddUpdates = blocksUpdated.filter {update => val blockId = update.blockUpdatedInfo.blockId blockId.isRDD} - val blockLocs = rddUpdates.map{ update => + val blockLocs = rddUpdates.map { update => (update.blockUpdatedInfo.blockId.name, update.blockUpdatedInfo.blockManagerId)} val blocksToManagers = blockLocs.groupBy(_._1).mapValues(_.size) @@ -201,11 +201,11 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext // 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 numDataLocs = blocksUpdated.filter { update => val blockId = update.blockUpdatedInfo.blockId blockId.isInstanceOf[ShuffleDataBlockId] }.size - val numIndexLocs = blocksUpdated.filter{ update => + val numIndexLocs = blocksUpdated.filter { update => val blockId = update.blockUpdatedInfo.blockId blockId.isInstanceOf[ShuffleIndexBlockId] }.size diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 098fe7d2562bc..96551f200e4dc 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -325,8 +325,7 @@ class KubernetesSuite extends SparkFunSuite val result = checkPodReady(namespace, name) result shouldBe (true) } - // Look for the string that indicates we're good to trigger decom - // on the driver + // Look for the string that indicates we're good to trigger decom on the driver logDebug("Waiting for first collect...") Eventually.eventually(TIMEOUT, INTERVAL) { assert(kubernetesTestComponents.kubernetesClient From 4eaf4dc45b948d725822dafa392f0191a4fe3a61 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 3 Jun 2020 17:55:53 -0700 Subject: [PATCH 05/47] Saw a test failure which could come from us not having a reasonable timeout here --- .../deploy/k8s/integrationtest/KubernetesSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 96551f200e4dc..bb7126057bf60 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -380,9 +380,10 @@ class KubernetesSuite extends SparkFunSuite .get(0) driverPodChecker(driverPod) - // If we're testing decommissioning we delete all the executors, but we should have - // an executor at some point. - Eventually.eventually(patienceTimeout, patienceInterval) { + + // If we're testing decommissioning we an executors, but we should have an executor + // at some point. + Eventually.eventually(TIMEOUT, patienceInterval) { execPods.values.nonEmpty should be (true) } execWatcher.close() @@ -497,6 +498,6 @@ private[spark] object KubernetesSuite { val SPARK_DFS_READ_WRITE_TEST = "org.apache.spark.examples.DFSReadWriteTest" val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" - val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(3, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(1, Seconds)) } From fd3354c39983d0e746460b81b6613449d8698171 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:40:52 -0700 Subject: [PATCH 06/47] Improve error logging Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e59195343712f..cddce59568471 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1868,7 +1868,7 @@ private[spark] class BlockManager( 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") + logError(s"Error ${e} while waiting for block to migrate") } } } From 8054404d5354b5479bebb321e19b3ff8c96fcd74 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:41:51 -0700 Subject: [PATCH 07/47] cleanup Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index cddce59568471..56ab726e2fb7b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1898,7 +1898,7 @@ private[spark] class BlockManager( val currentPeerSet = migrationPeers.keys.toSet val deadPeers = currentPeerSet.&~(livePeerSet) val newPeers = livePeerSet.&~(currentPeerSet) - migrationPeers ++= newPeers.map{peer => + 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) From dcff412e21585e840c2142b41997bbacbfd67a2f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:43:28 -0700 Subject: [PATCH 08/47] cleanup Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 56ab726e2fb7b..8764239f2ff37 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1906,7 +1906,7 @@ private[spark] class BlockManager( (peer, runnable) } // A peer may have entered a decommissioning state, don't transfer any new blocks - deadPeers.foreach{peer => + deadPeers.foreach { peer => migrationPeers.get(peer).foreach(_.running = false) } } From dc9d6480797dcf49974939108ae4d46ab1b66c46 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:45:07 -0700 Subject: [PATCH 09/47] Add more info to debugging Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- .../org/apache/spark/storage/BlockManagerMasterEndpoint.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 8c9dd42995064..e538eeb4ed0d3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -505,7 +505,8 @@ class BlockManagerMasterEndpoint( mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) return true case _ => - logDebug(s"Unexpected shuffle block type ${blockId}") + logDebug(s"Unexpected shuffle block type ${blockId}" + + s"as ${blockId.getClass().getSimpleName()}") return false } } From 493a298b87b5b8931579d3d32ec8df6a9e89da89 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:45:34 -0700 Subject: [PATCH 10/47] logging string interpolation Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 43370ee31abde..c6e04148427b7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -128,7 +128,7 @@ private class ShuffleStatus(numPartitions: Int) extends Logging { val mapStatusOpt = mapStatuses.find(_.mapId == mapId) mapStatusOpt match { case Some(mapStatus) => - logInfo("Updating map output for ${mapId} to ${bmAddress}") + logInfo(s"Updating map output for ${mapId} to ${bmAddress}") mapStatus.updateLocation(bmAddress) invalidateSerializedMapOutputStatusCache() case None => From c9139ef5f310014e3de0e36b70097283fb4e9e56 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:45:45 -0700 Subject: [PATCH 11/47] logging string interpolation Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- .../org/apache/spark/storage/BlockManagerMasterEndpoint.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index e538eeb4ed0d3..06cc607a10eb1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -501,7 +501,7 @@ class BlockManagerMasterEndpoint( logDebug("Received shuffle index block update for ${shuffleId} ${mapId}") return true case ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) => - logInfo("Received shuffle data block update for ${shuffleId} ${mapId}, performing update") + logInfo(s"Received shuffle data block update for ${shuffleId} ${mapId}, performing update") mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) return true case _ => From 0eef6fa72cb227fe8561de63df77d02f93492ba1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:46:59 -0700 Subject: [PATCH 12/47] logging string interpolation Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index c6e04148427b7..7a33237f71d2b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -132,7 +132,7 @@ private class ShuffleStatus(numPartitions: Int) extends Logging { mapStatus.updateLocation(bmAddress) invalidateSerializedMapOutputStatusCache() case None => - logError("Asked to update map output ${mapId} for untracked map status.") + logError(s"Asked to update map output ${mapId} for untracked map status.") } } From b50de4ec5628f88538384bed86c4ac19da7c0024 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:39:40 -0700 Subject: [PATCH 13/47] Generalize the decom put to check put as stream and shuffle blocks as well (help avoid cascading block migration) --- .../scala/org/apache/spark/storage/BlockManager.scala | 9 +++++++-- ...lockSavedOnDecommissionedBlockManagerException.scala} | 4 ++-- 2 files changed, 9 insertions(+), 4 deletions(-) rename core/src/main/scala/org/apache/spark/storage/{RDDBlockSavedOnDecommissionedBlockManagerException.scala => BlockSavedOnDecommissionedBlockManagerException.scala} (83%) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8764239f2ff37..55e73722757b1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -662,6 +662,11 @@ private[spark] class BlockManager( blockId: BlockId, level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { + + if (blockManagerDecommissioning) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + } + if (blockId.isShuffle || blockId.isInternalShuffle) { logInfo(s"Putting shuffle block ${blockId}") try { @@ -1309,8 +1314,8 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") - if (blockManagerDecommissioning && blockId.isRDD) { - throw new RDDBlockSavedOnDecommissionedBlockManagerException(blockId.asRDDId.get) + if (blockManagerDecommissioning) { + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } val putBlockInfo = { diff --git a/core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala rename to core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala index e6cef4dcc5e38..4684d9c67754d 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDBlockSavedOnDecommissionedBlockManagerException.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockSavedOnDecommissionedBlockManagerException.scala @@ -17,5 +17,5 @@ package org.apache.spark.storage -class RDDBlockSavedOnDecommissionedBlockManagerException(blockId: RDDBlockId) - extends Exception(s"RDD Block $blockId cannot be saved on decommissioned executor") +class BlockSavedOnDecommissionedBlockManagerException(blockId: BlockId) + extends Exception(s"Block $blockId cannot be saved on decommissioned executor") From e2d1057996bd5705cc1e4ede29ea0cc62ee841f1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:48:50 -0700 Subject: [PATCH 14/47] spacing --- .../apache/spark/storage/BlockManagerDecommissionSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala index 864a2a490b2ba..3e10f0603e30c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala @@ -187,7 +187,7 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext eventually(timeout(15.seconds), interval(10.milliseconds)) { if (persist) { // One of our blocks should have moved. - val rddUpdates = blocksUpdated.filter {update => + val rddUpdates = blocksUpdated.filter { update => val blockId = update.blockUpdatedInfo.blockId blockId.isRDD} val blockLocs = rddUpdates.map { update => From 31dc8360ee133a061d99585a08488754faaf54bf Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 10:57:38 -0700 Subject: [PATCH 15/47] Fix long line, make our shuffle block threads stop so we don't leak threads during testing. --- .../scala/org/apache/spark/storage/BlockManager.scala | 10 +++++++++- .../spark/storage/BlockManagerMasterEndpoint.scala | 4 ++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 55e73722757b1..c958b9cb1c82e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1907,7 +1907,6 @@ private[spark] class BlockManager( 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) } // A peer may have entered a decommissioning state, don't transfer any new blocks @@ -1916,6 +1915,14 @@ private[spark] class BlockManager( } } + + /** + * Stop migrating shuffle blocks. + */ + def stopOffloadingShuffleBlocks(): Unit = { + migrationPeers.values.foreach(_.running = false) + } + /** * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers * Visible for testing @@ -2089,6 +2096,7 @@ private[spark] class BlockManager( stopped = true logInfo("Stopping block replication thread") blockMigrationThread.interrupt() + stopOffloadingShuffleBlocks() } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 06cc607a10eb1..1986b62e0ed19 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -498,10 +498,10 @@ class BlockManagerMasterEndpoint( 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}") + logDebug("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}, performing update") + logInfo(s"Received shuffle data block update for ${shuffleId} ${mapId}, updating.") mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) return true case _ => From a2d5e649b1a386d74520be7de780f0d7145c0c57 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 11:05:44 -0700 Subject: [PATCH 16/47] Remove un-needed shuffleStatus.invalidateSerializedMapOutputStatusCache and log the scheduler when asked to decom and we can't --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 1 - core/src/main/scala/org/apache/spark/SparkContext.scala | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7a33237f71d2b..1e4b9906e0137 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -501,7 +501,6 @@ private[spark] class MapOutputTrackerMaster( shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => shuffleStatus.updateMapOutput(mapId, bmAddress) - shuffleStatus.invalidateSerializedMapOutputStatusCache() case None => logError(s"Asked to update map output for unknown shuffle ${shuffleId}") } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c36bbe9b0810f..0d4151f29c855 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1731,7 +1731,8 @@ class SparkContext(config: SparkConf) extends Logging { case b: CoarseGrainedSchedulerBackend => executorIds.foreach(b.decommissionExecutor) case _ => - logWarning("Decommissioning executors is not supported by current scheduler.") + logWarning(s"Decommissioning executors is not supported by current scheduler" + + s"${schedulerBackend}") } } From 20655a49f248bde7540f9b2545fa71a37beddeb7 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 11:19:14 -0700 Subject: [PATCH 17/47] Always transfer shuffle blocks as put, take out the spark.network.maxRemoteBlockSizeFetchToMem test that we don't need anymore, add back in submitting the thread I accidently took out in applying some CR feedback (a little fast on the ctrl-k) --- .../network/netty/NettyBlockTransferService.scala | 3 ++- .../scala/org/apache/spark/storage/BlockManager.scala | 3 ++- .../spark/storage/BlockManagerDecommissionSuite.scala | 10 +--------- 3 files changed, 5 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3de7377f99202..2069f812e067e 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -168,7 +168,8 @@ 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) + val asStream = (blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) || + blockId.isInternalShuffle || blockId.isShuffle) val callback = new RpcResponseCallback { override def onSuccess(response: ByteBuffer): Unit = { logTrace(s"Successfully uploaded block $blockId${if (asStream) " as stream" else ""}") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c958b9cb1c82e..6043f5b8c5721 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -664,7 +664,7 @@ private[spark] class BlockManager( classTag: ClassTag[_]): StreamCallbackWithID = { if (blockManagerDecommissioning) { - throw new BlockSavedOnDecommissionedBlockManagerException(blockId) + throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } if (blockId.isShuffle || blockId.isInternalShuffle) { @@ -1907,6 +1907,7 @@ private[spark] class BlockManager( 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) } // A peer may have entered a decommissioning state, don't transfer any new blocks diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala index 3e10f0603e30c..c92cc26678483 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala @@ -41,10 +41,6 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext runDecomTest(true, false, true) } - test(s"verify that shuffle blocks are migrated with force to disk") { - runDecomTest(false, true, false, remoteBlockSize = "1") - } - test(s"verify that shuffle blocks are migrated") { runDecomTest(false, true, false) } @@ -53,8 +49,7 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext runDecomTest(true, true, false) } - private def runDecomTest(persist: Boolean, shuffle: Boolean, migrateDuring: Boolean, - remoteBlockSize: String = "100000") = { + 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) @@ -66,9 +61,6 @@ class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext // workload we need to worry about. .set(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL, 1L) - // Allow force fetching to local disk - conf.set("spark.network.maxRemoteBlockSizeFetchToMem", remoteBlockSize) - sc = new SparkContext(master, "test", conf) // Create input RDD with 10 partitions From 5c131ef5369fae40cda1aa57abf24a01d3ffad5b Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 15:23:19 -0700 Subject: [PATCH 18/47] Logging and thread name cleanup --- .../org/apache/spark/storage/BlockManager.scala | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6043f5b8c5721..32943deeb8adf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1891,9 +1891,7 @@ private[spark] class BlockManager( // 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 @@ -2055,14 +2053,14 @@ private[spark] class BlockManager( try { // 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") + logDebug("Attempting to replicate all shuffle blocks") offloadShuffleBlocks() - logInfo(s"Done starting workers to migrate shuffle blocks") + logInfo("Done starting workers to migrate shuffle blocks") } if (conf.get(config.STORAGE_RDD_DECOMMISSION_ENABLED)) { - logDebug(s"Attempting to replicate all cached RDD blocks") + logDebug("Attempting to replicate all cached RDD blocks") decommissionRddCacheBlocks() - logInfo(s"Attempt to replicate all cached blocks done") + logInfo("Attempt to replicate all cached blocks done") } if (!conf.get(config.STORAGE_RDD_DECOMMISSION_ENABLED) && !conf.get(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED)) { @@ -2085,17 +2083,17 @@ private[spark] class BlockManager( } } blockMigrationThread.setDaemon(true) - blockMigrationThread.setName("block-replication-thread") + blockMigrationThread.setName("block-migration-thread") def start(): Unit = { - logInfo("Starting block replication thread") + logInfo("Starting block migration thread") blockMigrationThread.start() } def stop(): Unit = { if (!stopped) { stopped = true - logInfo("Stopping block replication thread") + logInfo("Stopping block migration thread") blockMigrationThread.interrupt() stopOffloadingShuffleBlocks() } From 5e11f1b8323169062bbd3f5039263e32677dbc65 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jun 2020 15:27:43 -0700 Subject: [PATCH 19/47] Try and fix up the K8s integration test a bit --- .../spark/deploy/k8s/integrationtest/DecommissionSuite.scala | 2 +- .../kubernetes/integration-tests/tests/decommissioning.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index e71d9ea127d25..4afca18b00b83 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -34,7 +34,7 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => // Ensure we have somewhere to migrate our data too .set("spark.executor.instances", "3") // The default of 30 seconds is fine, but for testing we just want to get this done fast. - .set("spark.storage.decommission.replicationReattemptInterval", "1s") + .set("spark.storage.decommission.replicationReattemptInterval", "1") .set("spark.storage.decommission.rdd_blocks", "true") runSparkApplicationAndVerifyCompletion( diff --git a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py index 84dc514b8988b..d34e61611461c 100644 --- a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py +++ b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py @@ -45,11 +45,11 @@ def addToAcc(x): print("1st accumulator value is: " + str(acc.value)) print("Waiting to give nodes time to finish migration, decom exec 1.") print("...") - time.sleep(5) + time.sleep(30) rdd.count() print("Waiting some more, please kill exec 1.") print("...") - time.sleep(5) + time.sleep(30) print("Executor node should be deleted now") rdd.count() rdd.collect() From 57965c89e95903b99b81f2769c380628cfd4049a Mon Sep 17 00:00:00 2001 From: Attila Zsolt Piros Date: Fri, 5 Jun 2020 16:55:05 +0200 Subject: [PATCH 20/47] add shuffle migration test to BlockManagerSuite --- .../shuffle/IndexShuffleBlockResolver.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 46 ++++++++++++++++++- 2 files changed, 45 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 7886e02b8b10d..cacce7b429285 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -47,7 +47,7 @@ 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) extends ShuffleBlockResolver with Logging with MigratableResolver { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 4143129810aa3..ea5f317d43b13 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.File import java.nio.ByteBuffer +import java.nio.file.Files import scala.collection.JavaConverters._ import scala.collection.mutable @@ -50,7 +51,7 @@ import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, Transpo import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.RpcEnv -import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} +import org.apache.spark.scheduler.{LiveListenerBus, MapStatus, SparkListenerBlockUpdated} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -100,7 +101,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, transferService: Option[BlockTransferService] = Option.empty, - testConf: Option[SparkConf] = None): BlockManager = { + testConf: Option[SparkConf] = None, + shuffleManager: SortShuffleManager = shuffleManager): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) @@ -1764,6 +1766,46 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) } + test("test migration of shuffle blocks during decommissioning") { + val shuffleManager1 = new SortShuffleManager(new SparkConf(false)) + val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1) + shuffleManager1.shuffleBlockResolver._blockManager = bm1 + + val shuffleManager2 = new SortShuffleManager(new SparkConf(false)) + 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) + 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) + + bm1.offloadShuffleBlocks() + + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) + } + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleData).toPath()) + === shuffleDataBlockContent) + assert(Files.readAllBytes(bm2.diskBlockManager.getFile(shuffleIndex).toPath()) + === shuffleIndexBlockContent) + } finally { + mapOutputTracker.unregisterShuffle(0) + } + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 var tempFileManager: DownloadFileManager = null From 93a8a4daa8a69ceb68fbaa422fed193bf4e52a83 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 8 Jun 2020 10:16:27 -0700 Subject: [PATCH 21/47] Use StorageLevel.DISK_ONLY instead of manually making our own, and store the execution service being used to run the threads so we can stop them explicitly. Still uses a graceful stop for other executors which become no longer healthy targets instead of a thread kill. --- .../shuffle/IndexShuffleBlockResolver.scala | 9 +------- .../apache/spark/storage/BlockManager.scala | 21 ++++++++----------- 2 files changed, 10 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index cacce7b429285..7a0db584126a6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -214,14 +214,7 @@ private[spark] class IndexShuffleBlockResolver( throw new IOException(s"fail to rename file ${fileTmp} to ${file}") } } - blockManager.reportBlockStatus(blockId, BlockStatus( - StorageLevel( - useDisk = true, - useMemory = false, - useOffHeap = false, - deserialized = false, - replication = 0) - , 0, diskSize)) + blockManager.reportBlockStatus(blockId, BlockStatus(StorageLevel.DISK_ONLY, 0, diskSize)) } override def onFailure(streamId: String, cause: Throwable): Unit = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 32943deeb8adf..7ac123d531983 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference} import java.nio.ByteBuffer import java.nio.channels.Channels import java.util.Collections -import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} +import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, ExecutorService, TimeUnit} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -1826,12 +1826,6 @@ private[spark] class BlockManager( @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 { @@ -1857,7 +1851,7 @@ private[spark] class BlockManager( peer.executorId, blockId, buffer, - storageLevel, + StorageLevel.DISK_ONLY, null)// class tag, we don't need for shuffle logInfo(s"Migrated sub block ${blockId}") } @@ -1879,7 +1873,9 @@ private[spark] class BlockManager( } } - private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() + private val migrationPeers = mutable.HashMap[ + BlockManagerId, + (ShuffleMigrationRunnable, ExecutorService)]() /** * Tries to offload all shuffle blocks that are registered with the shuffle service locally. @@ -1906,11 +1902,11 @@ private[spark] class BlockManager( val executor = ThreadUtils.newDaemonSingleThreadExecutor(s"migrate-shuffle-to-${peer}") val runnable = new ShuffleMigrationRunnable(peer) executor.submit(runnable) - (peer, 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(_.running = false) + migrationPeers.get(peer).foreach(_._1.running = false) } } @@ -1919,7 +1915,8 @@ private[spark] class BlockManager( * Stop migrating shuffle blocks. */ def stopOffloadingShuffleBlocks(): Unit = { - migrationPeers.values.foreach(_.running = false) + migrationPeers.values.foreach(_._1.running = false) + migrationPeers.values.foreach(_._2.shutdownNow()) } /** From 4560ebba907105f7aca203a72f2fed00d5128f01 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 8 Jun 2020 12:18:01 -0700 Subject: [PATCH 22/47] Cleanup thread leaks --- .../org/apache/spark/MapOutputTracker.scala | 7 +++- .../apache/spark/storage/BlockManager.scala | 41 ++++++++++++++----- .../spark/storage/BlockManagerSuite.scala | 30 ++++++++++++-- 3 files changed, 63 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1e4b9906e0137..7a6af20b7464c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -802,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 => + logError("Could not tell tracker we are stopping.", e) + } trackerEndpoint = null shuffleStatuses.clear() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 7ac123d531983..97df90eefb032 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -244,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 + @volatile private var decommissionManager: Option[BlockManagerDecommissionManager] = None // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. @@ -1829,7 +1829,7 @@ private[spark] class BlockManager( 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) { + while (running && !Thread.interrupted()) { val migrating = Option(shufflesToMigrate.poll()) migrating match { case None => @@ -1882,6 +1882,7 @@ private[spark] class BlockManager( * 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. */ def offloadShuffleBlocks(): Unit = { // Update the queue of shuffles to be migrated @@ -1915,8 +1916,14 @@ private[spark] class BlockManager( * Stop migrating shuffle blocks. */ def stopOffloadingShuffleBlocks(): Unit = { - migrationPeers.values.foreach(_._1.running = false) - migrationPeers.values.foreach(_._2.shutdownNow()) + logInfo("Stopping offloading shuffle blocks.") + // Stop as gracefully as possible. + migrationPeers.values.foreach{case (runnable, service) => + runnable.running = false} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdown()} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdownNow()} } /** @@ -2036,7 +2043,10 @@ private[spark] class BlockManager( */ private class BlockManagerDecommissionManager(conf: SparkConf) { @volatile private var stopped = false - private val blockMigrationThread = new Thread { + private lazy val blockMigrationExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") + + private val blockMigrationRunnable = new Runnable { val sleepInterval = conf.get( config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) @@ -2079,21 +2089,30 @@ private[spark] class BlockManager( } } } - blockMigrationThread.setDaemon(true) - blockMigrationThread.setName("block-migration-thread") def start(): Unit = { logInfo("Starting block migration thread") - blockMigrationThread.start() + blockMigrationExecutor.submit(blockMigrationRunnable) } def stop(): Unit = { if (!stopped) { stopped = true - logInfo("Stopping block migration thread") - blockMigrationThread.interrupt() + } + try { + blockMigrationExecutor.shutdown() + } catch { + case e: Exception => + logInfo(s"Error during shutdown ${e}") + } + try { stopOffloadingShuffleBlocks() + } catch { + case e: Exception => + logInfo(s"Error during shuffle shutdown ${e}") } + logInfo("Stopping block migration thread") + blockMigrationExecutor.shutdownNow() } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ea5f317d43b13..10711ee1e2a8e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -61,7 +61,7 @@ import org.apache.spark.util.io.ChunkedByteBuffer class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach with PrivateMethodTester with LocalSparkContext with ResetSystemProperties - with EncryptionFunSuite with TimeLimits { + with EncryptionFunSuite with TimeLimits with BeforeAndAfterAll { import BlockManagerSuite._ @@ -70,6 +70,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE var conf: SparkConf = null val allStores = ArrayBuffer[BlockManager]() + val sortShuffleManagers = ArrayBuffer[SortShuffleManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null var liveListenerBus: LiveListenerBus = null @@ -96,6 +97,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) } + private def makeSortShuffleManager(): SortShuffleManager = { + val newMgr = new SortShuffleManager(new SparkConf(false)) + sortShuffleManagers += newMgr + newMgr + } + private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, @@ -161,19 +168,34 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { + logInfo("Stopping...") conf = null allStores.foreach(_.stop()) allStores.clear() + sortShuffleManagers.foreach(_.stop()) + sortShuffleManagers.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null master = null liveListenerBus = null + logInfo("After each finished") } finally { super.afterEach() } } + override def afterAll(): Unit = { + try { + // Cleanup the reused items. + Option(bcastManager).foreach(_.stop()) + Option(mapOutputTracker).foreach(_.stop()) + Option(shuffleManager).foreach(_.stop()) + } finally { + super.afterAll() + } + } + private def stopBlockManager(blockManager: BlockManager): Unit = { allStores -= blockManager blockManager.stop() @@ -1767,11 +1789,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("test migration of shuffle blocks during decommissioning") { - val shuffleManager1 = new SortShuffleManager(new SparkConf(false)) + val shuffleManager1 = makeSortShuffleManager() val bm1 = makeBlockManager(3500, "exec1", shuffleManager = shuffleManager1) shuffleManager1.shuffleBlockResolver._blockManager = bm1 - val shuffleManager2 = new SortShuffleManager(new SparkConf(false)) + val shuffleManager2 = makeSortShuffleManager() val bm2 = makeBlockManager(3500, "exec2", shuffleManager = shuffleManager2) shuffleManager2.shuffleBlockResolver._blockManager = bm2 @@ -1803,6 +1825,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE === shuffleIndexBlockContent) } finally { mapOutputTracker.unregisterShuffle(0) + // Avoid thread leak + bm1.stopOffloadingShuffleBlocks() } } From 13aaa4951fa951b81a5628063ce6592719d6af61 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 8 Jun 2020 12:18:33 -0700 Subject: [PATCH 23/47] Remove excess logging --- .../test/scala/org/apache/spark/storage/BlockManagerSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 10711ee1e2a8e..4930ee2676056 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -168,7 +168,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { - logInfo("Stopping...") conf = null allStores.foreach(_.stop()) allStores.clear() @@ -179,7 +178,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE rpcEnv = null master = null liveListenerBus = null - logInfo("After each finished") } finally { super.afterEach() } From 7af749263f01c566ac25879551a9d8f88b85a1c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 9 Jun 2020 09:38:33 +0200 Subject: [PATCH 24/47] Increase cohesion by moving methods and members into BlockManagerDecommissionManager --- .../apache/spark/storage/BlockManager.scala | 295 +++++++++--------- .../spark/storage/BlockManagerSuite.scala | 11 +- 2 files changed, 158 insertions(+), 148 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 97df90eefb032..bc1f81280ca9a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -186,6 +186,9 @@ private[spark] class BlockManager( private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES) + private val maxReplicationFailuresForDecommission = + conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) + val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. val deleteFilesOnStop = @@ -256,7 +259,7 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None - private lazy val migratableResolver: MigratableResolver = { + private[storage] lazy val migratableResolver: MigratableResolver = { shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] } @@ -1822,151 +1825,23 @@ 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 - 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 = - 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.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("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, ExecutorService)]() - - /** - * 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. - */ - def offloadShuffleBlocks(): Unit = { - // Update the queue of shuffles to be migrated - logInfo("Offloading shuffle blocks") - val localShuffles = 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 = 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) - } - } + private[storage] def getMigratableRDDBlocks(): Seq[ReplicateBlock] = + master.getReplicateInfoForRDDBlocks(blockManagerId) - - /** - * Stop migrating shuffle blocks. - */ - def stopOffloadingShuffleBlocks(): Unit = { - logInfo("Stopping offloading shuffle blocks.") - // Stop as gracefully as possible. - migrationPeers.values.foreach{case (runnable, service) => - runnable.running = false} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdown()} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdownNow()} - } - - /** - * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers - * Visible for testing - */ - def decommissionRddCacheBlocks(): Unit = { - val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) - - if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + - "for block manager decommissioning") + private[storage] def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { + val replicatedSuccessfully = replicateBlock( + blockToReplicate.blockId, + blockToReplicate.replicas.toSet, + blockToReplicate.maxReplicas, + maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) + if (replicatedSuccessfully) { + logInfo(s"Block ${blockToReplicate.blockId} offloaded successfully, Removing block now") + removeBlock(blockToReplicate.blockId) + logInfo(s"Block ${blockToReplicate.blockId} removed") } else { - logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") - return - } - - // Maximum number of storage replication failure which replicateBlock can handle - val maxReplicationFailures = conf.get( - config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) - - // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) - // so that we end up prioritize them over each other - val blocksFailedReplication = replicateBlocksInfo.map { - case ReplicateBlock(blockId, existingReplicas, maxReplicas) => - val replicatedSuccessfully = replicateBlock( - blockId, - existingReplicas.toSet, - maxReplicas, - maxReplicationFailures = Some(maxReplicationFailures)) - if (replicatedSuccessfully) { - logInfo(s"Block $blockId offloaded successfully, Removing block now") - removeBlock(blockId) - logInfo(s"Block $blockId removed") - } else { - logWarning(s"Failed to offload block $blockId") - } - (blockId, replicatedSuccessfully) - }.filterNot(_._2).map(_._1) - if (blocksFailedReplication.nonEmpty) { - logWarning("Blocks failed replication in cache decommissioning " + - s"process: ${blocksFailedReplication.mkString(",")}") + logWarning(s"Failed to offload block ${blockToReplicate.blockId}") } + replicatedSuccessfully } /** @@ -2041,11 +1916,143 @@ private[spark] class BlockManager( * Class to handle block manager decommissioning retries * It creates a Thread to retry offloading all RDD cache blocks */ - private class BlockManagerDecommissionManager(conf: SparkConf) { + private[storage] class BlockManagerDecommissionManager(conf: SparkConf) { + 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 = + 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.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, " + + s"adding ${shuffleMap} back to migration queue") + shufflesToMigrate.add(shuffleMap) + case None => + logError(s"Error ${e} while waiting for block to migrate") + } + } + } + } + @volatile private var stopped = false + + private val migrationPeers = + mutable.HashMap[BlockManagerId, (ShuffleMigrationRunnable, ExecutorService)]() + private lazy val blockMigrationExecutor = ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") + /** + * 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. + */ + def offloadShuffleBlocks(): Unit = { + // Update the queue of shuffles to be migrated + logInfo("Offloading shuffle blocks") + val localShuffles = 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 = peerProvider.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) + } + } + + + /** + * Stop migrating shuffle blocks. + */ + def stopOffloadingShuffleBlocks(): Unit = { + logInfo("Stopping offloading shuffle blocks.") + // Stop as gracefully as possible. + migrationPeers.values.foreach{case (runnable, service) => + runnable.running = false} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdown()} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdownNow()} + } + + /** + * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers + * Visible for testing + */ + def decommissionRddCacheBlocks(): Unit = { + val replicateBlocksInfo = getMigratableRDDBlocks() + + if (replicateBlocksInfo.nonEmpty) { + logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + + "for block manager decommissioning") + } else { + logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") + return + } + + // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) + // so that we end up prioritize them over each other + val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => + val replicatedSuccessfully = migrateBlock(replicateBlock) + (replicateBlock.blockId, replicatedSuccessfully) + }.filterNot(_._2).map(_._1) + if (blocksFailedReplication.nonEmpty) { + logWarning("Blocks failed replication in cache decommissioning " + + s"process: ${blocksFailedReplication.mkString(",")}") + } + } + private val blockMigrationRunnable = new Runnable { val sleepInterval = conf.get( config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 4930ee2676056..bc111d1ae7185 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -139,6 +139,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManager } + private def decommissionManager(bm: BlockManager) = new bm.BlockManagerDecommissionManager(conf) + override def beforeEach(): Unit = { super.beforeEach() // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case @@ -1759,7 +1761,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).contains(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + decommissionManager(store1).decommissionRddCacheBlocks() assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).toSet === Set(store2.blockManagerId, store3.blockManagerId)) @@ -1779,7 +1781,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId)) - store1.decommissionRddCacheBlocks() + decommissionManager(store1).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 @@ -1804,6 +1806,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) mapOutputTracker.registerShuffle(0, 1) + val decomManager = decommissionManager(bm1) try { mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0)) assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId) @@ -1812,7 +1815,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(env.conf).thenReturn(conf) SparkEnv.set(env) - bm1.offloadShuffleBlocks() + decomManager.offloadShuffleBlocks() eventually(timeout(1.second), interval(10.milliseconds)) { assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) @@ -1824,7 +1827,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } finally { mapOutputTracker.unregisterShuffle(0) // Avoid thread leak - bm1.stopOffloadingShuffleBlocks() + decomManager.stopOffloadingShuffleBlocks() } } From a7d9238a9b7f71374f5f4858365f9ef00a58d3e9 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 9 Jun 2020 12:47:34 -0700 Subject: [PATCH 25/47] Remove ref to peerProvider we didn't port over --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index bc1f81280ca9a..d706b8029e396 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1994,7 +1994,7 @@ private[spark] class BlockManager( // Update the threads doing migrations // TODO: Sort & only start as many threads as min(||blocks||, ||targets||) using location pref - val livePeerSet = peerProvider.getPeers(false).toSet + val livePeerSet = getPeers(false).toSet val currentPeerSet = migrationPeers.keys.toSet val deadPeers = currentPeerSet.&~(livePeerSet) val newPeers = livePeerSet.&~(currentPeerSet) From 953e5f20b6569307fd6f3ea41e7a2577a702cb7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 9 Jun 2020 10:39:27 +0200 Subject: [PATCH 26/47] extract BlockManagerDecommissionManager into a toplevel class --- .../apache/spark/storage/BlockManager.scala | 225 +--------------- .../BlockManagerDecommissionManager.scala | 255 ++++++++++++++++++ .../spark/storage/BlockManagerSuite.scala | 7 +- 3 files changed, 268 insertions(+), 219 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d706b8029e396..5da20d6f09d7b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference} import java.nio.ByteBuffer import java.nio.channels.Channels import java.util.Collections -import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, ExecutorService, TimeUnit} +import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -263,11 +263,6 @@ private[spark] class BlockManager( shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] } - // 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)]() - /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -1818,7 +1813,12 @@ private[spark] class BlockManager( if (!blockManagerDecommissioning) { logInfo("Starting block manager decommissioning process...") blockManagerDecommissioning = true - decommissionManager = Some(new BlockManagerDecommissionManager(conf)) + decommissionManager = Some(new BlockManagerDecommissionManager( + conf, + blockTransferService, + this, + migratableResolver, + peerProvider)) decommissionManager.foreach(_.start()) } else { logDebug("Block manager already in decommissioning state") @@ -1912,217 +1912,6 @@ private[spark] class BlockManager( data.dispose() } - /** - * Class to handle block manager decommissioning retries - * It creates a Thread to retry offloading all RDD cache blocks - */ - private[storage] class BlockManagerDecommissionManager(conf: SparkConf) { - 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 = - 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.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, " + - s"adding ${shuffleMap} back to migration queue") - shufflesToMigrate.add(shuffleMap) - case None => - logError(s"Error ${e} while waiting for block to migrate") - } - } - } - } - - @volatile private var stopped = false - - private val migrationPeers = - mutable.HashMap[BlockManagerId, (ShuffleMigrationRunnable, ExecutorService)]() - - private lazy val blockMigrationExecutor = - ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") - - /** - * 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. - */ - def offloadShuffleBlocks(): Unit = { - // Update the queue of shuffles to be migrated - logInfo("Offloading shuffle blocks") - val localShuffles = 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 = 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) - } - } - - - /** - * Stop migrating shuffle blocks. - */ - def stopOffloadingShuffleBlocks(): Unit = { - logInfo("Stopping offloading shuffle blocks.") - // Stop as gracefully as possible. - migrationPeers.values.foreach{case (runnable, service) => - runnable.running = false} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdown()} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdownNow()} - } - - /** - * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers - * Visible for testing - */ - def decommissionRddCacheBlocks(): Unit = { - val replicateBlocksInfo = getMigratableRDDBlocks() - - if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + - "for block manager decommissioning") - } else { - logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") - return - } - - // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) - // so that we end up prioritize them over each other - val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => - val replicatedSuccessfully = migrateBlock(replicateBlock) - (replicateBlock.blockId, replicatedSuccessfully) - }.filterNot(_._2).map(_._1) - if (blocksFailedReplication.nonEmpty) { - logWarning("Blocks failed replication in cache decommissioning " + - s"process: ${blocksFailedReplication.mkString(",")}") - } - } - - private val blockMigrationRunnable = new Runnable { - val sleepInterval = conf.get( - config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) - - override def run(): Unit = { - var failures = 0 - while (blockManagerDecommissioning - && !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) - } - } - } - } - - def start(): Unit = { - logInfo("Starting block migration thread") - blockMigrationExecutor.submit(blockMigrationRunnable) - } - - def stop(): Unit = { - if (!stopped) { - stopped = true - } - try { - blockMigrationExecutor.shutdown() - } catch { - case e: Exception => - logInfo(s"Error during shutdown ${e}") - } - try { - stopOffloadingShuffleBlocks() - } catch { - case e: Exception => - logInfo(s"Error during shuffle shutdown ${e}") - } - logInfo("Stopping block migration thread") - blockMigrationExecutor.shutdownNow() - } - } - def stop(): Unit = { decommissionManager.foreach(_.stop()) blockTransferService.close() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala new file mode 100644 index 0000000000000..9ed3c833fd34c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala @@ -0,0 +1,255 @@ +/* + * 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.network.BlockTransferService +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( + conf: SparkConf, + blockTransferService: BlockTransferService, + migratableRDDResolver: MigratableRDDResolver, + migratableShuffleBlockResolver: MigratableResolver, + peerProvider: BlockManagerPeerProvider) extends Logging { + + // 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 + 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 = + migratableShuffleBlockResolver.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.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, " + + s"adding ${shuffleMap} back to migration queue") + shufflesToMigrate.add(shuffleMap) + case None => + logError(s"Error ${e} while waiting for block to migrate") + } + } + } + } + + @volatile private var stopped = false + + private val migrationPeers = + mutable.HashMap[BlockManagerId, (ShuffleMigrationRunnable, ExecutorService)]() + + private lazy val blockMigrationExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") + + /** + * 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. + */ + def offloadShuffleBlocks(): Unit = { + // Update the queue of shuffles to be migrated + logInfo("Offloading shuffle blocks") + val localShuffles = migratableShuffleBlockResolver.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 = peerProvider.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) + } + } + + + /** + * Stop migrating shuffle blocks. + */ + def stopOffloadingShuffleBlocks(): Unit = { + logInfo("Stopping offloading shuffle blocks.") + // Stop as gracefully as possible. + migrationPeers.values.foreach{case (runnable, service) => + runnable.running = false} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdown()} + migrationPeers.values.foreach{case (runnable, service) => + service.shutdownNow()} + } + + /** + * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers + * Visible for testing + */ + def decommissionRddCacheBlocks(): Unit = { + val replicateBlocksInfo = migratableRDDResolver.getMigratableRDDBlocks() + + if (replicateBlocksInfo.nonEmpty) { + logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + + "for block manager decommissioning") + } else { + logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") + return + } + + // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) + // so that we end up prioritize them over each other + val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => + val replicatedSuccessfully = migratableRDDResolver.migrateBlock(replicateBlock) + (replicateBlock.blockId, replicatedSuccessfully) + }.filterNot(_._2).map(_._1) + if (blocksFailedReplication.nonEmpty) { + logWarning("Blocks failed replication in cache decommissioning " + + s"process: ${blocksFailedReplication.mkString(",")}") + } + } + + 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) + } + } + } + } + + def start(): Unit = { + logInfo("Starting block migration thread") + blockMigrationExecutor.submit(blockMigrationRunnable) + } + + def stop(): Unit = { + if (!stopped) { + stopped = true + } + try { + blockMigrationExecutor.shutdown() + } catch { + case e: Exception => + logInfo(s"Error during shutdown ${e}") + } + try { + stopOffloadingShuffleBlocks() + } catch { + case e: Exception => + logInfo(s"Error during shuffle shutdown ${e}") + } + logInfo("Stopping block migration thread") + blockMigrationExecutor.shutdownNow() + } +} + diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index bc111d1ae7185..d8588aab33f67 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -139,7 +139,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManager } - private def decommissionManager(bm: BlockManager) = new bm.BlockManagerDecommissionManager(conf) + private def decommissionManager(bm: BlockManager) = new BlockManagerDecommissionManager( + conf, + bm.blockTransferService, + bm, + bm.migratableResolver, + bm.peerProvider) override def beforeEach(): Unit = { super.beforeEach() From d63ca07959e1347790e3cd86489e0dc21f0eb034 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 9 Jun 2020 14:30:08 -0700 Subject: [PATCH 27/47] Cleanup to just use the BM not the other interfaces --- .../apache/spark/storage/BlockManager.scala | 28 ++------------ .../BlockManagerDecommissionManager.scala | 38 +++++++++++++------ .../spark/storage/BlockManagerSuite.scala | 15 +++----- 3 files changed, 35 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5da20d6f09d7b..29fb1241d4846 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -186,9 +186,6 @@ private[spark] class BlockManager( private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES) - private val maxReplicationFailuresForDecommission = - conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) - val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. val deleteFilesOnStop = @@ -259,6 +256,8 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver + // for shuffles. Used in BlockManagerDecommissionManager & block puts. private[storage] lazy val migratableResolver: MigratableResolver = { shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] } @@ -1570,7 +1569,7 @@ private[spark] class BlockManager( /** * Get peer block managers in the system. */ - private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + private[storage] def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { val cachedPeersTtl = conf.get(config.STORAGE_CACHED_PEERS_TTL) // milliseconds val diff = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastPeerFetchTimeNs) @@ -1815,10 +1814,7 @@ private[spark] class BlockManager( blockManagerDecommissioning = true decommissionManager = Some(new BlockManagerDecommissionManager( conf, - blockTransferService, - this, - migratableResolver, - peerProvider)) + this)) decommissionManager.foreach(_.start()) } else { logDebug("Block manager already in decommissioning state") @@ -1828,22 +1824,6 @@ private[spark] class BlockManager( private[storage] def getMigratableRDDBlocks(): Seq[ReplicateBlock] = master.getReplicateInfoForRDDBlocks(blockManagerId) - private[storage] def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { - val replicatedSuccessfully = replicateBlock( - blockToReplicate.blockId, - blockToReplicate.replicas.toSet, - blockToReplicate.maxReplicas, - maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) - if (replicatedSuccessfully) { - logInfo(s"Block ${blockToReplicate.blockId} offloaded successfully, Removing block now") - removeBlock(blockToReplicate.blockId) - logInfo(s"Block ${blockToReplicate.blockId} removed") - } else { - logWarning(s"Failed to offload block ${blockToReplicate.blockId}") - } - replicatedSuccessfully - } - /** * Remove all blocks belonging to the given broadcast. */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala index 9ed3c833fd34c..dfbe9beeb7356 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala @@ -26,7 +26,6 @@ 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.network.BlockTransferService import org.apache.spark.shuffle.MigratableResolver import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock import org.apache.spark.util.ThreadUtils @@ -37,10 +36,10 @@ import org.apache.spark.util.ThreadUtils */ private[storage] class BlockManagerDecommissionManager( conf: SparkConf, - blockTransferService: BlockTransferService, - migratableRDDResolver: MigratableRDDResolver, - migratableShuffleBlockResolver: MigratableResolver, - peerProvider: BlockManagerPeerProvider) extends Logging { + bm: BlockManager) extends Logging { + + private val maxReplicationFailuresForDecommission = + conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) // Shuffles which are either in queue for migrations or migrated private val migratingShuffles = mutable.HashSet[(Int, Long)]() @@ -67,11 +66,11 @@ private[storage] class BlockManagerDecommissionManager( case Some((shuffleId, mapId)) => logInfo(s"Trying to migrate shuffle ${shuffleId},${mapId} to ${peer}") val blocks = - migratableShuffleBlockResolver.getMigrationBlocks(shuffleId, mapId) + bm.migratableResolver.getMigrationBlocks(shuffleId, mapId) logInfo(s"Got migration sub-blocks ${blocks}") blocks.foreach { case (blockId, buffer) => logInfo(s"Migrating sub-block ${blockId}") - blockTransferService.uploadBlockSync( + bm.blockTransferService.uploadBlockSync( peer.host, peer.port, peer.executorId, @@ -118,14 +117,14 @@ private[storage] class BlockManagerDecommissionManager( def offloadShuffleBlocks(): Unit = { // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") - val localShuffles = migratableShuffleBlockResolver.getStoredShuffles() + 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 = peerProvider.getPeers(false).toSet + val livePeerSet = bm.getPeers(false).toSet val currentPeerSet = migrationPeers.keys.toSet val deadPeers = currentPeerSet.&~(livePeerSet) val newPeers = livePeerSet.&~(currentPeerSet) @@ -162,7 +161,7 @@ private[storage] class BlockManagerDecommissionManager( * Visible for testing */ def decommissionRddCacheBlocks(): Unit = { - val replicateBlocksInfo = migratableRDDResolver.getMigratableRDDBlocks() + val replicateBlocksInfo = bm.getMigratableRDDBlocks() if (replicateBlocksInfo.nonEmpty) { logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + @@ -175,7 +174,7 @@ private[storage] class BlockManagerDecommissionManager( // TODO: We can sort these blocks based on some policy (LRU/blockSize etc) // so that we end up prioritize them over each other val blocksFailedReplication = replicateBlocksInfo.map { replicateBlock => - val replicatedSuccessfully = migratableRDDResolver.migrateBlock(replicateBlock) + val replicatedSuccessfully = migrateBlock(replicateBlock) (replicateBlock.blockId, replicatedSuccessfully) }.filterNot(_._2).map(_._1) if (blocksFailedReplication.nonEmpty) { @@ -184,6 +183,22 @@ private[storage] class BlockManagerDecommissionManager( } } + private def migrateBlock(blockToReplicate: ReplicateBlock): Boolean = { + val replicatedSuccessfully = bm.replicateBlock( + blockToReplicate.blockId, + blockToReplicate.replicas.toSet, + blockToReplicate.maxReplicas, + maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) + if (replicatedSuccessfully) { + logInfo(s"Block ${blockToReplicate.blockId} offloaded successfully, Removing block now") + bm.removeBlock(blockToReplicate.blockId) + logInfo(s"Block ${blockToReplicate.blockId} removed") + } else { + logWarning(s"Failed to offload block ${blockToReplicate.blockId}") + } + replicatedSuccessfully + } + private val blockMigrationRunnable = new Runnable { val sleepInterval = conf.get( config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) @@ -252,4 +267,3 @@ private[storage] class BlockManagerDecommissionManager( blockMigrationExecutor.shutdownNow() } } - diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d8588aab33f67..fcd900700cc77 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -139,13 +139,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManager } - private def decommissionManager(bm: BlockManager) = new BlockManagerDecommissionManager( - conf, - bm.blockTransferService, - bm, - bm.migratableResolver, - bm.peerProvider) - override def beforeEach(): Unit = { super.beforeEach() // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case @@ -1766,7 +1759,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).contains(store1.blockManagerId)) - decommissionManager(store1).decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissionManager(conf, store1) + decomManager.decommissionRddCacheBlocks() assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).toSet === Set(store2.blockManagerId, store3.blockManagerId)) @@ -1786,7 +1780,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId)) - decommissionManager(store1).decommissionRddCacheBlocks() + val decomManager = new BlockManagerDecommissionManager(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 @@ -1811,7 +1806,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) mapOutputTracker.registerShuffle(0, 1) - val decomManager = decommissionManager(bm1) + val decomManager = new BlockManagerDecommissionManager(conf, bm1) try { mapOutputTracker.registerMapOutput(0, 0, MapStatus(bm1.blockManagerId, Array(blockSize), 0)) assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm1.blockManagerId) From 197a5246273c4f7bac491a710f4eebcf5651b6e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 9 Jun 2020 14:56:17 +0200 Subject: [PATCH 28/47] Some reordering and adding BlockManagerDecommissionManagerSuite --- .../BlockManagerDecommissionManager.scala | 106 +++++++++--------- ...BlockManagerDecommissionManagerSuite.scala | 90 +++++++++++++++ 2 files changed, 141 insertions(+), 55 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala index dfbe9beeb7356..874ef0f311e5d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala @@ -41,12 +41,6 @@ private[storage] class BlockManagerDecommissionManager( private val maxReplicationFailuresForDecommission = conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) - // 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 = { @@ -89,8 +83,7 @@ private[storage] class BlockManagerDecommissionManager( case e: Exception => migrating match { case Some(shuffleMap) => - logError(s"Error ${e} during migration, " + - s"adding ${shuffleMap} back to migration queue") + 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") @@ -99,6 +92,13 @@ private[storage] class BlockManagerDecommissionManager( } } + // 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 = @@ -107,6 +107,46 @@ private[storage] class BlockManagerDecommissionManager( 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 @@ -114,7 +154,7 @@ private[storage] class BlockManagerDecommissionManager( * Requires an Indexed based shuffle resolver. * Note: if called in testing please call stopOffloadingShuffleBlocks to avoid thread leakage. */ - def offloadShuffleBlocks(): Unit = { + private[storage] def offloadShuffleBlocks(): Unit = { // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") val localShuffles = bm.migratableResolver.getStoredShuffles() @@ -141,11 +181,10 @@ private[storage] class BlockManagerDecommissionManager( } } - /** * Stop migrating shuffle blocks. */ - def stopOffloadingShuffleBlocks(): Unit = { + private[storage] def stopOffloadingShuffleBlocks(): Unit = { logInfo("Stopping offloading shuffle blocks.") // Stop as gracefully as possible. migrationPeers.values.foreach{case (runnable, service) => @@ -160,7 +199,7 @@ private[storage] class BlockManagerDecommissionManager( * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers * Visible for testing */ - def decommissionRddCacheBlocks(): Unit = { + private[storage] def decommissionRddCacheBlocks(): Unit = { val replicateBlocksInfo = bm.getMigratableRDDBlocks() if (replicateBlocksInfo.nonEmpty) { @@ -199,49 +238,6 @@ private[storage] class BlockManagerDecommissionManager( replicatedSuccessfully } - 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) - } - } - } - } - def start(): Unit = { logInfo("Starting block migration thread") blockMigrationExecutor.submit(blockMigrationRunnable) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala new file mode 100644 index 0000000000000..cd798dd78cdc2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.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 BlockManagerDecommissionManagerSuite 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( + sparkConf, + bm) + + bmDecomManager.start() + + eventually(timeout(5.second), interval(10.milliseconds)) { + assert(bmDecomManager.shufflesToMigrate.isEmpty == true) + verify(bm, times(1)).replicateBlock( + mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) + verify(blockTransferService, times(2)) + .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), + mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + } + } +} From 5855eb4b81f5c6aff6b260c43fe41a34d97590dc Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 9 Jun 2020 15:25:08 -0700 Subject: [PATCH 29/47] Rename the two different test suites for decommissioning so the difference is clear from the class name (integration-style/unit-style) --- ...ite.scala => BlockManagerDecommissionIntegrationSuite.scala} | 2 +- ...nagerSuite.scala => BlockManagerDecommissionUnitSuite.scala} | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename core/src/test/scala/org/apache/spark/storage/{BlockManagerDecommissionSuite.scala => BlockManagerDecommissionIntegrationSuite.scala} (99%) rename core/src/test/scala/org/apache/spark/storage/{BlockManagerDecommissionManagerSuite.scala => BlockManagerDecommissionUnitSuite.scala} (97%) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala rename to core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index c92cc26678483..ca19b138edd35 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.util.{ResetSystemProperties, ThreadUtils} -class BlockManagerDecommissionSuite extends SparkFunSuite with LocalSparkContext +class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties with Eventually { val numExecs = 3 diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index cd798dd78cdc2..89b513d0c2cf5 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.shuffle.MigratableResolver import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock -class BlockManagerDecommissionManagerSuite extends SparkFunSuite with Matchers { +class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { private val bmPort = 12345 From c3f8658a49ac622e75a7e69fc9bdab7530861792 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 10 Jun 2020 10:11:18 -0700 Subject: [PATCH 30/47] Various small cleanups including renaming BlockManagerDecommissionManager to BlockManagerDecommissioner --- .../apache/spark/storage/BlockManager.scala | 29 +++++++++---------- ...scala => BlockManagerDecommissioner.scala} | 6 ++-- .../BlockManagerDecommissionUnitSuite.scala | 4 +-- .../spark/storage/BlockManagerSuite.scala | 6 ++-- 4 files changed, 20 insertions(+), 25 deletions(-) rename core/src/main/scala/org/apache/spark/storage/{BlockManagerDecommissionManager.scala => BlockManagerDecommissioner.scala} (98%) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 29fb1241d4846..310d1e3d968f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -244,8 +244,7 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ - @volatile private var blockManagerDecommissioning: Boolean = false - @volatile private var decommissionManager: Option[BlockManagerDecommissionManager] = None + @volatile private var decommissioner: Option[BlockManagerDecommissioner] = None // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. @@ -257,7 +256,7 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver - // for shuffles. Used in BlockManagerDecommissionManager & block puts. + // for shuffles. Used in BlockManagerDecommissioner & block puts. private[storage] lazy val migratableResolver: MigratableResolver = { shuffleManager.shuffleBlockResolver.asInstanceOf[MigratableResolver] } @@ -660,7 +659,7 @@ private[spark] class BlockManager( level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { - if (blockManagerDecommissioning) { + if (decommissioner.isDefined) { throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } @@ -1311,7 +1310,7 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") - if (blockManagerDecommissioning) { + if (decommissioner.isDefined) { throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } @@ -1808,16 +1807,14 @@ private[spark] class BlockManager( blocksToRemove.size } - def decommissionBlockManager(): Unit = { - if (!blockManagerDecommissioning) { - logInfo("Starting block manager decommissioning process...") - blockManagerDecommissioning = true - decommissionManager = Some(new BlockManagerDecommissionManager( - conf, - this)) - decommissionManager.foreach(_.start()) - } else { - logDebug("Block manager already in decommissioning state") + def decommissionBlockManager(): Unit = synchronized { + decommissioner match { + case None => + logInfo("Starting block manager decommissioning process...") + decommissioner = Some(new BlockManagerDecommissioner(conf, this)) + decommissioner.foreach(_.start()) + case Some(_) => + logDebug("Block manager already in decommissioning state") } } @@ -1893,7 +1890,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - decommissionManager.foreach(_.stop()) + decommissioner.foreach(_.stop()) blockTransferService.close() if (blockStoreClient ne blockTransferService) { // Closing should be idempotent, but maybe not for the NioBlockTransferService. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 874ef0f311e5d..f346442ecbffb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissionManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -34,9 +34,9 @@ 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( - conf: SparkConf, - bm: BlockManager) extends Logging { +private[storage] class BlockManagerDecommissioner( + conf: SparkConf, + bm: BlockManager) extends Logging { private val maxReplicationFailuresForDecommission = conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 89b513d0c2cf5..87909432ab651 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -72,9 +72,7 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { when(bm.getMigratableRDDBlocks()) .thenReturn(Seq(storedBlock1)) - val bmDecomManager = new BlockManagerDecommissionManager( - sparkConf, - bm) + val bmDecomManager = new BlockManagerDecommissioner(sparkConf, bm) bmDecomManager.start() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index fcd900700cc77..24d853e5657ff 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1759,7 +1759,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).contains(store1.blockManagerId)) - val decomManager = new BlockManagerDecommissionManager(conf, store1) + val decomManager = new BlockManagerDecommissioner(conf, store1) decomManager.decommissionRddCacheBlocks() assert(master.getLocations(blockId).size === 2) assert(master.getLocations(blockId).toSet === Set(store2.blockManagerId, @@ -1780,7 +1780,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations(blockIdLarge) === Seq(store1.blockManagerId)) assert(master.getLocations(blockIdSmall) === Seq(store1.blockManagerId)) - val decomManager = new BlockManagerDecommissionManager(conf, store1) + val decomManager = new BlockManagerDecommissioner(conf, store1) decomManager.decommissionRddCacheBlocks() // Smaller block offloaded to store2 assert(master.getLocations(blockIdSmall) === Seq(store2.blockManagerId)) @@ -1806,7 +1806,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE Files.write(bm1.diskBlockManager.getFile(shuffleIndex).toPath(), shuffleIndexBlockContent) mapOutputTracker.registerShuffle(0, 1) - val decomManager = new BlockManagerDecommissionManager(conf, bm1) + 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) From 206a3c3e786159007cb142b1e9aa2f45d0fa4077 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 12 Jun 2020 11:23:20 -0700 Subject: [PATCH 31/47] Stop the block decom manager at the end of the block decom unit test --- .../BlockManagerDecommissionUnitSuite.scala | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 87909432ab651..56b6d18f2fe2c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -74,15 +74,19 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { val bmDecomManager = new BlockManagerDecommissioner(sparkConf, bm) - bmDecomManager.start() - - eventually(timeout(5.second), interval(10.milliseconds)) { - assert(bmDecomManager.shufflesToMigrate.isEmpty == true) - verify(bm, times(1)).replicateBlock( - mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) - verify(blockTransferService, times(2)) - .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), - mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + try { + bmDecomManager.start() + + eventually(timeout(5.second), interval(10.milliseconds)) { + assert(bmDecomManager.shufflesToMigrate.isEmpty == true) + verify(bm, times(1)).replicateBlock( + mc.eq(storedBlockId1), mc.any(), mc.any(), mc.eq(Some(3))) + verify(blockTransferService, times(2)) + .uploadBlockSync(mc.eq("host2"), mc.eq(bmPort), mc.eq("exec2"), mc.any(), mc.any(), + mc.eq(StorageLevel.DISK_ONLY), mc.isNull()) + } + } finally { + bmDecomManager.stop() } } } From f45a89c89f3e2998fdcb65c4c30a237e5e3ddd66 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 12 Jun 2020 19:26:09 -0700 Subject: [PATCH 32/47] Add an explanation about why we use producer/consumer here. --- .../storage/BlockManagerDecommissioner.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index f346442ecbffb..17bcdf2621602 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -41,6 +41,22 @@ private[storage] class BlockManagerDecommissioner( 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 = { From 4d7b5b8c3d3d4af755a1253f4a5c72482416ec14 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 12 Jun 2020 20:15:48 -0700 Subject: [PATCH 33/47] Add a case class for the shuffle block id + map id so we're more clear on what were passing around than just rabndom tuples --- .../shuffle/IndexShuffleBlockResolver.scala | 8 ++++-- .../spark/shuffle/MigratableResolver.scala | 4 +-- .../spark/shuffle/ShuffleBlockInfo.scala | 28 +++++++++++++++++++ .../storage/BlockManagerDecommissioner.scala | 16 +++++------ .../BlockManagerDecommissionUnitSuite.scala | 6 ++-- 5 files changed, 46 insertions(+), 16 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 7a0db584126a6..357d5ea8bd701 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -61,7 +61,7 @@ private[spark] class IndexShuffleBlockResolver( /** * Get the shuffle files that are stored locally. Used for block migrations. */ - override def getStoredShuffles(): Set[(Int, Long)] = { + override def getStoredShuffles(): Set[ShuffleBlockInfo] = { // Matches ShuffleIndexBlockId name val pattern = "shuffle_(\\d+)_(\\d+)_.+\\.index".r val rootDirs = blockManager.diskBlockManager.localDirs @@ -71,7 +71,7 @@ private[spark] class IndexShuffleBlockResolver( logDebug(s"Got block files ${filenames.toList}") filenames.flatMap { fname => pattern.findAllIn(fname).matchData.map { - matched => (matched.group(1).toInt, matched.group(2).toLong) + matched => ShuffleBlockInfo(matched.group(1).toInt, matched.group(2).toLong) } }.toSet } @@ -229,7 +229,9 @@ private[spark] class IndexShuffleBlockResolver( /** * Get the index & data block for migration. */ - def getMigrationBlocks(shuffleId: Int, mapId: Long): List[(BlockId, ManagedBuffer)] = { + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] = { + val shuffleId = shuffleBlockInfo.shuffleId + val mapId = shuffleBlockInfo.mapId // Load the index block val indexFile = getIndexFile(shuffleId, mapId) val indexBlockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID) diff --git a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala index 768d4f8db4364..891c8282d26de 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala @@ -32,7 +32,7 @@ trait MigratableResolver { /** * Get the shuffle ids that are stored locally. Used for block migrations. */ - def getStoredShuffles(): Set[(Int, Long)] + def getStoredShuffles(): Set[ShuffleBlockInfo] /** * Write a provided shuffle block as a stream. Used for block migrations. @@ -43,5 +43,5 @@ trait MigratableResolver { /** * Get the blocks for migration for a particular shuffle and map. */ - def getMigrationBlocks(shuffleId: Int, mapId: Long): List[(BlockId, ManagedBuffer)] + def getMigrationBlocks(shuffleBlockInfo: ShuffleBlockInfo): List[(BlockId, ManagedBuffer)] } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala new file mode 100644 index 0000000000000..99ceee81d349d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockInfo.scala @@ -0,0 +1,28 @@ +/* + * 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 + +/** + * :: Experimental :: + * An experimental case class used by MigratableResolver to return the shuffleId and mapId in a + * type safe way. + */ +@Experimental +case class ShuffleBlockInfo(shuffleId: Int, mapId: Long) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 17bcdf2621602..7c57d6791a452 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -26,7 +26,7 @@ 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.shuffle.{MigratableResolver, ShuffleBlockInfo} import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock import org.apache.spark.util.ThreadUtils @@ -60,7 +60,7 @@ private[storage] class BlockManagerDecommissioner( private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable { @volatile var running = true override def run(): Unit = { - var migrating: Option[(Int, Long)] = None + 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 { @@ -73,10 +73,10 @@ private[storage] class BlockManagerDecommissioner( // 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}") + case Some(shuffleBlockInfo) => + logInfo(s"Trying to migrate shuffle ${shuffleBlockInfo} to ${peer}") val blocks = - bm.migratableResolver.getMigrationBlocks(shuffleId, mapId) + bm.migratableResolver.getMigrationBlocks(shuffleBlockInfo) logInfo(s"Got migration sub-blocks ${blocks}") blocks.foreach { case (blockId, buffer) => logInfo(s"Migrating sub-block ${blockId}") @@ -90,7 +90,7 @@ private[storage] class BlockManagerDecommissioner( null)// class tag, we don't need for shuffle logInfo(s"Migrated sub block ${blockId}") } - logInfo(s"Migrated ${shuffleId},${mapId} to ${peer}") + logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}") } } // This catch is intentionally outside of the while running block. @@ -109,11 +109,11 @@ private[storage] class BlockManagerDecommissioner( } // Shuffles which are either in queue for migrations or migrated - private val migratingShuffles = mutable.HashSet[(Int, Long)]() + private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]() // Shuffles which are queued for migration private[storage] val shufflesToMigrate = - new java.util.concurrent.ConcurrentLinkedQueue[(Int, Long)]() + new java.util.concurrent.ConcurrentLinkedQueue[ShuffleBlockInfo]() @volatile private var stopped = false diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 56b6d18f2fe2c..929fd9527a9f4 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -28,7 +28,7 @@ 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.shuffle.{MigratableResolver, ShuffleBlockInfo} import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { @@ -44,10 +44,10 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { ids: Set[(Int, Long, Int)]): Unit = { when(mockMigratableShuffleResolver.getStoredShuffles()) - .thenReturn(ids.map(triple => (triple._1, triple._2)).toSet) + .thenReturn(ids.map(triple => ShuffleBlockInfo(triple._1, triple._2)).toSet) ids.foreach { case (shuffleId: Int, mapId: Long, reduceId: Int) => - when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any(), mc.any())) + when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any())) .thenReturn(List( (ShuffleIndexBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])), (ShuffleDataBlockId(shuffleId, mapId, reduceId), mock(classOf[ManagedBuffer])))) From d23cbbfc1f7368dc9c74c5e9cf0350c541e96d56 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 12 Jun 2020 22:53:51 -0700 Subject: [PATCH 34/47] Fix the unit test --- .../spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index b4092be466e27..71f4d13cb8175 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -27,7 +27,7 @@ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleBlockInfo} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -83,7 +83,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(dataFile.exists()) assert(dataFile.length() === 30) assert(!dataTmp.exists()) - assert(storedShuffles === Set((1, 2))) + assert(storedShuffles === Set(ShuffleBlockInfo(1, 2))) val lengths2 = new Array[Long](3) val dataTmp2 = File.createTempFile("shuffle", null, tempDir) From e5731162be3875331818d5894871341425274607 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 15 Jun 2020 14:28:12 -0700 Subject: [PATCH 35/47] Code review feedback: add test for rejecting blocks with bad shuffle resolver, fix string interopolation, remove un-needed retry logic in decommissioning, etc. --- .../org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 11 -------- .../spark/internal/config/package.scala | 6 ++--- .../apache/spark/storage/BlockManager.scala | 4 +-- .../storage/BlockManagerDecommissioner.scala | 26 +++++++++---------- .../spark/storage/BlockManagerMaster.scala | 1 + .../storage/BlockManagerMasterEndpoint.scala | 5 ++-- .../spark/storage/BlockManagerSuite.scala | 16 +++++++++++- 8 files changed, 37 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7a6af20b7464c..af00891ff0073 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -805,7 +805,7 @@ private[spark] class MapOutputTrackerMaster( try { sendTracker(StopMapOutputTracker) } catch { - case e: Exception => + case e: SparkException => logError("Could not tell tracker we are stopping.", e) } trackerEndpoint = null diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0d4151f29c855..b2a86e1bec294 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1725,17 +1725,6 @@ 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(s"Decommissioning executors is not supported by current scheduler" + - s"${schedulerBackend}") - } - } - /** The version of Spark on which this application is running. */ def version: String = SPARK_VERSION diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 009f38aa0574a..70ef2dcfb4bcd 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -421,15 +421,15 @@ package object config { .createWithDefault(false) private[spark] val STORAGE_SHUFFLE_DECOMMISSION_ENABLED = - ConfigBuilder("spark.storage.decommission.shuffle_blocks") + ConfigBuilder("spark.storage.decommission.shuffleBlocks.enabled") .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " + - "an indexed shuffle resolver (like sort based shuffe)") + "a migratable shuffle resolver (like sort based shuffe)") .version("3.1.0") .booleanConf .createWithDefault(false) private[spark] val STORAGE_RDD_DECOMMISSION_ENABLED = - ConfigBuilder("spark.storage.decommission.rdd_blocks") + ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") .doc("Whether to transfer RDD blocks during block manager decommissioning.") .version("3.1.0") .booleanConf diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 310d1e3d968f2..3803a226bc70a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -371,7 +371,7 @@ private[spark] class BlockManager( ThreadUtils.awaitReady(replicationFuture, Duration.Inf) } catch { case NonFatal(t) => - throw new Exception("Error occurred while waiting for replication to finish", t) + throw new SparkException("Error occurred while waiting for replication to finish", t) } } if (blockWasSuccessfullyStored) { @@ -668,7 +668,7 @@ private[spark] class BlockManager( try { return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) } catch { - case e: ClassCastException => throw new Exception( + case e: ClassCastException => throw new SparkException( s"Unexpected shuffle block ${blockId} with unsupported shuffle " + s"resolver ${shuffleManager.shuffleBlockResolver}") } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 7c57d6791a452..fcee8c4684f82 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -68,7 +68,7 @@ private[storage] class BlockManagerDecommissioner( val migrating = Option(shufflesToMigrate.poll()) migrating match { case None => - logInfo("Nothing to migrate") + 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 @@ -88,7 +88,7 @@ private[storage] class BlockManagerDecommissioner( buffer, StorageLevel.DISK_ONLY, null)// class tag, we don't need for shuffle - logInfo(s"Migrated sub block ${blockId}") + logDebug(s"Migrated sub block ${blockId}") } logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}") } @@ -127,8 +127,14 @@ private[storage] class BlockManagerDecommissioner( val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) override def run(): Unit = { - var failures = 0 - while (!stopped && !Thread.interrupted() && failures < 20) { + 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") + 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. @@ -142,12 +148,6 @@ private[storage] class BlockManagerDecommissioner( 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 { @@ -155,9 +155,9 @@ private[storage] class BlockManagerDecommissioner( 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) + logError("Error occurred while trying to replicate for block manager decommissioning.", + e) + stopped = true } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index ff0b099e44b51..3cfa5d2a25818 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.storage import scala.collection.generic.CanBuildFrom diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 1986b62e0ed19..a864733a7549a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -336,14 +336,13 @@ class BlockManagerMasterEndpoint( val info = blockManagerInfo(blockManagerId) val rddBlocks = info.blocks.keySet().asScala.filter(_.isRDD) - val result = rddBlocks.map { blockId => + 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 } // Remove a block from the slaves that have it. This can only be used to remove @@ -498,7 +497,7 @@ class BlockManagerMasterEndpoint( 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.") + 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.") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 24d853e5657ff..c31c43c8b00d9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -54,6 +54,7 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.{LiveListenerBus, MapStatus, SparkListenerBlockUpdated} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.{ShuffleBlockResolver, ShuffleManager} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util._ @@ -109,7 +110,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master: BlockManagerMaster = this.master, transferService: Option[BlockTransferService] = Option.empty, testConf: Option[SparkConf] = None, - shuffleManager: SortShuffleManager = shuffleManager): BlockManager = { + shuffleManager: ShuffleManager = shuffleManager): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) @@ -1728,6 +1729,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE verify(liveListenerBus, never()).post(SparkListenerBlockUpdated(BlockUpdatedInfo(updateInfo))) } + test("we reject putting blocks when we have the wrong shuffle resolver") { + val badShuffleManager = mock(classOf[ShuffleManager]) + val badShuffleResolver = mock(classOf[ShuffleBlockResolver]) + when(badShuffleManager.shuffleBlockResolver).thenReturn(badShuffleResolver) + val shuffleBlockId = ShuffleDataBlockId(0, 0, 0) + val bm = makeBlockManager(100, "exec1", shuffleManager = badShuffleManager) + val message = "message" + val exception = intercept[SparkException] { + bm.putBlockDataAsStream(shuffleBlockId, StorageLevel.DISK_ONLY, ClassTag(message.getClass)) + } + assert(exception.getMessage.contains("unsupported shuffle resolver")) + } + test("test decommission block manager should not be part of peers") { val exec1 = "exec1" val exec2 = "exec2" From 9503ca56b699c7547fe345bf78cbf19c899fecaa Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 15 Jun 2020 14:41:16 -0700 Subject: [PATCH 36/47] Update warning message to use key from the config package --- .../org/apache/spark/storage/BlockManagerDecommissioner.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index fcee8c4684f82..882698c46f35f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -130,8 +130,8 @@ private[storage] class BlockManagerDecommissioner( 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") + s"${config.STORAGE_RDD_DECOMMISSION_ENABLED.key}\n" + + s"${config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED.key}") stopped = true } while (!stopped && !Thread.interrupted()) { From 968418ef7f6630d0e113b20a82c5b424329443ff Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 16 Jun 2020 11:16:07 -0700 Subject: [PATCH 37/47] Used a (configurable) thread pool for shuffle migrations to allow users to limit the number of concurrent migrations. We still limit to one runnable per peer executor so we don't overwhelm any particular target even at large values of the thread pool. --- .../spark/internal/config/package.scala | 9 ++++++++ .../storage/BlockManagerDecommissioner.scala | 21 +++++++++---------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 70ef2dcfb4bcd..1bb0903283cfb 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -428,6 +428,15 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val STORAGE_SHUFFLE_DECOMMISSION_MAX_THREADS = + ConfigBuilder("spark.storage.decommission.shuffleBlocks.maxThreads") + .doc("Maximum number of threads to use in migrating shuffle files.") + .version("3.1.0") + .intConf + .checkValue(_ > 0, "The maximum number of threads should be positive") + .createWithDefault(10) + + private[spark] val STORAGE_RDD_DECOMMISSION_ENABLED = ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") .doc("Whether to transfer RDD blocks during block manager decommissioning.") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 882698c46f35f..f02f501121fc7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -118,7 +118,7 @@ private[storage] class BlockManagerDecommissioner( @volatile private var stopped = false private val migrationPeers = - mutable.HashMap[BlockManagerId, (ShuffleMigrationRunnable, ExecutorService)]() + mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() private lazy val blockMigrationExecutor = ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") @@ -163,6 +163,9 @@ private[storage] class BlockManagerDecommissioner( } } + 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 @@ -186,14 +189,13 @@ private[storage] class BlockManagerDecommissioner( 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)) + shuffleMigrationPool.submit(runnable) + (peer, runnable) } // A peer may have entered a decommissioning state, don't transfer any new blocks deadPeers.foreach { peer => - migrationPeers.get(peer).foreach(_._1.running = false) + migrationPeers.get(peer).foreach(_.running = false) } } @@ -203,12 +205,9 @@ private[storage] class BlockManagerDecommissioner( private[storage] def stopOffloadingShuffleBlocks(): Unit = { logInfo("Stopping offloading shuffle blocks.") // Stop as gracefully as possible. - migrationPeers.values.foreach{case (runnable, service) => - runnable.running = false} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdown()} - migrationPeers.values.foreach{case (runnable, service) => - service.shutdownNow()} + migrationPeers.values.foreach{_.running = false} + shuffleMigrationPool.shutdown() + shuffleMigrationPool.shutdownNow() } /** From 2d61c41e3f2f44bb1445c71a5a6f0174df0d6692 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 17 Jun 2020 12:31:56 -0700 Subject: [PATCH 38/47] Fix the K8s decom test to use the correct config params --- .../deploy/k8s/integrationtest/DecommissionSuite.scala | 8 ++++---- .../deploy/k8s/integrationtest/KubernetesSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index 4afca18b00b83..9f6303c9b1378 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.deploy.k8s.integrationtest +import org.apache.spark.internal.config import org.apache.spark.internal.config.Worker private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => @@ -28,14 +29,13 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => .set(Worker.WORKER_DECOMMISSION_ENABLED.key, "true") .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.container.image", pyImage) - .set("spark.storage.decommission.enabled", "true") - .set("spark.storage.decommission.shuffle_blocks", "true") - .set("spark.storage.decommission.shuffle_blocks", "true") + .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true") + .set(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED.key, "true") + .set(config.STORAGE_RDD_DECOMMISSION_ENABLED.key, "true") // Ensure we have somewhere to migrate our data too .set("spark.executor.instances", "3") // The default of 30 seconds is fine, but for testing we just want to get this done fast. .set("spark.storage.decommission.replicationReattemptInterval", "1") - .set("spark.storage.decommission.rdd_blocks", "true") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_DECOMISSIONING, diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index bb7126057bf60..0ca3b701b248a 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -40,9 +40,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite - with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite + with BeforeAndAfterAll with BeforeAndAfter /* with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually + with DepsTestsSuite */ with DecommissionSuite /* with RTestsSuite */ with Logging with Eventually with Matchers { From ac096f46aa6e658704d7726efe7d66ece280b83e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 17 Jun 2020 13:51:40 -0700 Subject: [PATCH 39/47] Re-enable rest of k8s tests --- .../spark/deploy/k8s/integrationtest/KubernetesSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 0ca3b701b248a..bb7126057bf60 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -40,9 +40,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite - with BeforeAndAfterAll with BeforeAndAfter /* with BasicTestsSuite with SecretsTestsSuite + with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite */ with DecommissionSuite /* with RTestsSuite */ with Logging with Eventually + with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually with Matchers { From ebec1a7b2751ffae902be696b092cdfadbc43c5b Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 29 Jun 2020 12:44:42 -0700 Subject: [PATCH 40/47] Add a retry limit on shuffle blocks to match cache blocks during migration --- .../storage/BlockManagerDecommissioner.scala | 54 ++++++++++--------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index f02f501121fc7..97719fffc5cc3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -60,12 +60,12 @@ private[storage] class BlockManagerDecommissioner( private class ShuffleMigrationRunnable(peer: BlockManagerId) extends Runnable { @volatile var running = true override def run(): Unit = { - var migrating: Option[ShuffleBlockInfo] = None + 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()) { - val migrating = Option(shufflesToMigrate.poll()) + migrating = Option(shufflesToMigrate.poll()) migrating match { case None => logDebug("Nothing to migrate") @@ -73,24 +73,28 @@ private[storage] class BlockManagerDecommissioner( // 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}") + 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}") } - logInfo(s"Migrated ${shuffleBlockInfo} to ${peer}") } } // This catch is intentionally outside of the while running block. @@ -98,11 +102,11 @@ private[storage] class BlockManagerDecommissioner( } 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 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 ${e} while waiting for block to migrate") + logError(s"Error while waiting for block to migrate", e) } } } @@ -113,7 +117,7 @@ private[storage] class BlockManagerDecommissioner( // Shuffles which are queued for migration private[storage] val shufflesToMigrate = - new java.util.concurrent.ConcurrentLinkedQueue[ShuffleBlockInfo]() + new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() @volatile private var stopped = false @@ -178,7 +182,7 @@ private[storage] class BlockManagerDecommissioner( logInfo("Offloading shuffle blocks") val localShuffles = bm.migratableResolver.getStoredShuffles() val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq - shufflesToMigrate.addAll(newShufflesToMigrate.asJava) + shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) migratingShuffles ++= newShufflesToMigrate // Update the threads doing migrations From d97c6ee17ee6d80e91067e714c5aeedfd8be9e35 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 29 Jun 2020 15:23:44 -0700 Subject: [PATCH 41/47] CR feedback, make the config names more consistent, change the default number of threads to match the other similar config, add some comments, make isShuffle treat internalShuffle's the same simplifying the code some, seperate the shuffle block refresh and rdd block threads, rever un-needed change to SparkContext, and other misc cleanups --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/internal/config/package.scala | 8 +- .../netty/NettyBlockTransferService.scala | 4 +- .../apache/spark/scheduler/MapStatus.scala | 13 +- .../shuffle/IndexShuffleBlockResolver.scala | 2 +- .../org/apache/spark/storage/BlockId.scala | 8 +- .../apache/spark/storage/BlockManager.scala | 10 +- .../storage/BlockManagerDecommissioner.scala | 112 ++++++++++++------ .../storage/BlockManagerMasterEndpoint.scala | 6 +- .../apache/spark/storage/BlockIdSuite.scala | 4 +- ...kManagerDecommissionIntegrationSuite.scala | 4 +- .../BlockManagerDecommissionUnitSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 2 +- .../integrationtest/DecommissionSuite.scala | 4 +- 14 files changed, 118 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b2a86e1bec294..38d7319b1f0ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -57,7 +57,7 @@ import org.apache.spark.resource._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.shuffle.ShuffleDataIOUtils import org.apache.spark.shuffle.api.ShuffleDriverComponents diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 1bb0903283cfb..3a2010af460f8 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -420,7 +420,7 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val STORAGE_SHUFFLE_DECOMMISSION_ENABLED = + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED = ConfigBuilder("spark.storage.decommission.shuffleBlocks.enabled") .doc("Whether to transfer shuffle blocks during block manager decommissioning. Requires " + "a migratable shuffle resolver (like sort based shuffe)") @@ -428,16 +428,16 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val STORAGE_SHUFFLE_DECOMMISSION_MAX_THREADS = + private[spark] val STORAGE_DECOMMISSION_SHUFFLE_MAX_THREADS = ConfigBuilder("spark.storage.decommission.shuffleBlocks.maxThreads") .doc("Maximum number of threads to use in migrating shuffle files.") .version("3.1.0") .intConf .checkValue(_ > 0, "The maximum number of threads should be positive") - .createWithDefault(10) + .createWithDefault(8) - private[spark] val STORAGE_RDD_DECOMMISSION_ENABLED = + private[spark] val STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED = ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") .doc("Whether to transfer RDD blocks during block manager decommissioning.") .version("3.1.0") diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 2069f812e067e..5d9cea068b097 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -168,8 +168,10 @@ private[spark] class NettyBlockTransferService( // Everything else is encoded using our binary protocol. val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag))) + // 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) || - blockId.isInternalShuffle || blockId.isShuffle) + blockId.isShuffle) val callback = new RpcResponseCallback { override def onSuccess(response: ByteBuffer): Unit = { logTrace(s"Successfully uploaded block $blockId${if (asStream) " as stream" else ""}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 9dee1f779bcb5..0af3a2e171906 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -30,13 +30,14 @@ import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the - * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. + * task has shuffle files stored on as well as the sizes of outputs for each reducer, for passing + * on to the reduce tasks. */ private[spark] sealed trait MapStatus { /** Location where this task output is. */ def location: BlockManagerId - def updateLocation(bm: BlockManagerId): Unit + def updateLocation(newLoc: BlockManagerId): Unit /** * Estimated size for the reduce block, in bytes. @@ -128,8 +129,8 @@ private[spark] class CompressedMapStatus( override def location: BlockManagerId = loc - override def updateLocation(bm: BlockManagerId): Unit = { - loc = bm + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc } override def getSizeForBlock(reduceId: Int): Long = { @@ -184,8 +185,8 @@ private[spark] class HighlyCompressedMapStatus private ( override def location: BlockManagerId = loc - override def updateLocation(bm: BlockManagerId): Unit = { - loc = bm + override def updateLocation(newLoc: BlockManagerId): Unit = { + loc = newLoc } override def getSizeForBlock(reduceId: Int): Long = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 357d5ea8bd701..6a77e2c805a06 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -184,7 +184,7 @@ private[spark] class IndexShuffleBlockResolver( case ShuffleDataBlockId(shuffleId, mapId, _) => getDataFile(shuffleId, mapId) case _ => - throw new Exception(s"Unexpected shuffle block transfer ${blockId} as " + + throw new IllegalStateException(s"Unexpected shuffle block transfer ${blockId} as " + s"${blockId.getClass().getSimpleName()}") } val fileTmp = Utils.tempFileWith(file) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 398db1f4875ee..7b084e73c92f9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -38,11 +38,11 @@ sealed abstract class BlockId { // convenience methods def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD: Boolean = isInstanceOf[RDDBlockId] - def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] - def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] - def isInternalShuffle: Boolean = { - isInstanceOf[ShuffleDataBlockId] || isInstanceOf[ShuffleIndexBlockId] + def isShuffle: Boolean = { + (isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] || + isInstanceOf[ShuffleDataBlockId] || isInstanceOf[ShuffleIndexBlockId]) } + def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] override def toString: String = name } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 3803a226bc70a..f1ba091525554 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -244,6 +244,7 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ + // This is volatile since if it's defined we should not accept remote blocks. @volatile private var decommissioner: Option[BlockManagerDecommissioner] = None // A DownloadFileManager used to track all the files of remote blocks which are above the @@ -255,6 +256,9 @@ private[spark] class BlockManager( var hostLocalDirManager: Option[HostLocalDirManager] = None + @inline final private def isDecommissioning() = { + decommissioner.isDefined + } // This is a lazy val so someone can migrating RDDs even if they don't have a MigratableResolver // for shuffles. Used in BlockManagerDecommissioner & block puts. private[storage] lazy val migratableResolver: MigratableResolver = { @@ -659,11 +663,11 @@ private[spark] class BlockManager( level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { - if (decommissioner.isDefined) { + if (isDecommissioning()) { throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } - if (blockId.isShuffle || blockId.isInternalShuffle) { + if (blockId.isShuffle) { logInfo(s"Putting shuffle block ${blockId}") try { return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) @@ -1310,7 +1314,7 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") - if (decommissioner.isDefined) { + if (isDecommissioning()) { throw new BlockSavedOnDecommissionedBlockManagerException(blockId) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 97719fffc5cc3..46f6763a9690c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -119,39 +119,26 @@ private[storage] class BlockManagerDecommissioner( private[storage] val shufflesToMigrate = new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() + // Set if we encounter an error attempting to migrate and stop. @volatile private var stopped = false private val migrationPeers = mutable.HashMap[BlockManagerId, ShuffleMigrationRunnable]() - private lazy val blockMigrationExecutor = - ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission") + private lazy val rddBlockMigrationExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-rdd") - private val blockMigrationRunnable = new Runnable { + private val rddBlockMigrationRunnable = 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 - } + assert(conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) 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") - } + 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 { @@ -167,9 +154,37 @@ private[storage] class BlockManagerDecommissioner( } } + private lazy val shuffleBlockMigrationRefreshExecutor = + ThreadUtils.newDaemonSingleThreadExecutor("block-manager-decommission-shuffle") + + private val shuffleBlockMigrationRefreshRunnable = new Runnable { + val sleepInterval = conf.get(config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) + + override def run() { + assert(conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) + while (!stopped && !Thread.interrupted()) { + try { + logDebug("Attempting to replicate all shuffle blocks") + refreshOffloadingShuffleBlocks() + logInfo("Done starting workers to migrate shuffle blocks") + 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)) + conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_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 @@ -177,7 +192,7 @@ private[storage] class BlockManagerDecommissioner( * Requires an Indexed based shuffle resolver. * Note: if called in testing please call stopOffloadingShuffleBlocks to avoid thread leakage. */ - private[storage] def offloadShuffleBlocks(): Unit = { + private[storage] def refreshOffloadingShuffleBlocks(): Unit = { // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") val localShuffles = bm.migratableResolver.getStoredShuffles() @@ -186,11 +201,10 @@ private[storage] class BlockManagerDecommissioner( 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) + val deadPeers = currentPeerSet.diff(livePeerSet) + val newPeers = livePeerSet.diff(currentPeerSet) migrationPeers ++= newPeers.map { peer => logDebug(s"Starting thread to migrate shuffle blocks to ${peer}") val runnable = new ShuffleMigrationRunnable(peer) @@ -259,26 +273,58 @@ private[storage] class BlockManagerDecommissioner( def start(): Unit = { logInfo("Starting block migration thread") - blockMigrationExecutor.submit(blockMigrationRunnable) + if (conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + rddBlockMigrationExecutor.submit(rddBlockMigrationRunnable) + } + if (conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) { + shuffleBlockMigrationRefreshExecutor.submit(shuffleBlockMigrationRefreshRunnable) + } + if (!conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED) && + !conf.get(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED)) { + logError(s"Storage decommissioning attempted but neither " + + s"${config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key} or " + + s"${config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key} is enabled ") + stopped = true + } } def stop(): Unit = { - if (!stopped) { + if (stopped) { + return + } else { stopped = true } try { - blockMigrationExecutor.shutdown() + rddBlockMigrationExecutor.shutdown() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdown() } catch { case e: Exception => - logInfo(s"Error during shutdown ${e}") + logError(s"Error during shutdown", e) } try { stopOffloadingShuffleBlocks() } catch { case e: Exception => - logInfo(s"Error during shuffle shutdown ${e}") + logError(s"Error during shutdown", e) + } + logInfo("Forcing block migrations threads to stop") + try { + rddBlockMigrationExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) + } + try { + shuffleBlockMigrationRefreshExecutor.shutdownNow() + } catch { + case e: Exception => + logError(s"Error during shutdown", e) } - logInfo("Stopping block migration thread") - blockMigrationExecutor.shutdownNow() + logInfo("Stopped storage decommissioner") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index a864733a7549a..915aeafc606a1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -491,16 +491,16 @@ class BlockManagerMasterEndpoint( storageLevel: StorageLevel, memSize: Long, diskSize: Long): Boolean = { - logInfo(s"Updating block info on master ${blockId} for ${blockManagerId}") + logDebug(s"Updating block info on master ${blockId} for ${blockManagerId}") - if (blockId.isInternalShuffle) { + if (blockId.isShuffle) { 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.") + logDebug(s"Received shuffle data block update for ${shuffleId} ${mapId}, updating.") mapOutputTracker.updateMapOutput(shuffleId, mapId, blockManagerId) return true case _ => diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index ef7b13875540f..d7009e6ed5799 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -87,7 +87,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -100,7 +100,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(!id.isShuffle) + assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index ca19b138edd35..1e5f93b18ad7c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -55,8 +55,8 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS val conf = new SparkConf().setAppName("test").setMaster(master) .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) .set(config.STORAGE_DECOMMISSION_ENABLED, true) - .set(config.STORAGE_RDD_DECOMMISSION_ENABLED, persist) - .set(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED, shuffle) + .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) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index 929fd9527a9f4..e56febfb95462 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -36,8 +36,8 @@ 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) + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED, true) + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED, true) private def registerShuffleBlocks( mockMigratableShuffleResolver: MigratableResolver, diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c31c43c8b00d9..a3a6db07c7f0d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1829,7 +1829,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(env.conf).thenReturn(conf) SparkEnv.set(env) - decomManager.offloadShuffleBlocks() + decomManager.refreshOffloadingShuffleBlocks() eventually(timeout(1.second), interval(10.milliseconds)) { assert(mapOutputTracker.shuffleStatuses(0).mapStatuses(0).location === bm2.blockManagerId) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala index 9f6303c9b1378..fd67a03160afb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -30,8 +30,8 @@ private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.container.image", pyImage) .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true") - .set(config.STORAGE_SHUFFLE_DECOMMISSION_ENABLED.key, "true") - .set(config.STORAGE_RDD_DECOMMISSION_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true") + .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true") // Ensure we have somewhere to migrate our data too .set("spark.executor.instances", "3") // The default of 30 seconds is fine, but for testing we just want to get this done fast. From 56a990316a947785b2502e3129c49ab00bef9b90 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 29 Jun 2020 15:31:10 -0700 Subject: [PATCH 42/47] Minor comment cleanup --- .../storage/BlockManagerDecommissionIntegrationSuite.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 1e5f93b18ad7c..4337df774263f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -148,7 +148,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS ThreadUtils.awaitResult(asyncCount, 15.seconds) } - // Decommission one of the executor + // 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}") @@ -157,13 +157,12 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS logDebug(s"Decommissioning executor ${execToDecommission}") sched.decommissionExecutor(execToDecommission) - // Wait for job to finish + // 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 + // All tasks finished, so accum should have been increased numParts times. assert(accum.value === numParts) - // All tasks should be successful, nothing should have failed sc.listenerBus.waitUntilEmpty() if (shuffle) { // mappers & reducers which succeeded From fe5ba7befc243a30377b0d3057ec3862726db2d3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 14 Jul 2020 18:29:48 -0700 Subject: [PATCH 43/47] Executor start time variance made this test a bit flaky. Waiting for all the executors to come up first only slows the entire suite down by ~4s and should remove that flake --- ...kManagerDecommissionIntegrationSuite.scala | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 4337df774263f..6a315805f6cb9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -63,17 +63,21 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS 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") - // Do a count to wait for the executors to be registered. 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(500) + Thread.sleep(1000) } accum.add(1) x.map(y => (y, y)) @@ -119,21 +123,9 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS testRdd.persist() } - // Wait for the first executor to start - TestUtils.waitUntilExecutorsUp(sc = sc, - numExecutors = 1, - timeout = 20000) // 20s - // Start the computation of RDD - this step will also cache the RDD val asyncCount = testRdd.countAsync() - // Wait for all of the executors to start - TestUtils.waitUntilExecutorsUp(sc = sc, - // We need to make sure there is the original plus one exec to migrate too, we don't need - // the full set. - numExecutors = 2, - timeout = 30000) // 30s - // Wait for the job to have started. taskStartSem.acquire(1) // Wait for each executor + driver to have it's broadcast info delivered. @@ -175,7 +167,7 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS } // Wait for our respective blocks to have migrated - eventually(timeout(15.seconds), interval(10.milliseconds)) { + eventually(timeout(30.seconds), interval(10.milliseconds)) { if (persist) { // One of our blocks should have moved. val rddUpdates = blocksUpdated.filter { update => From eb43f2055a38067c63f925526f91d435d7c90aaa Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 15 Jul 2020 09:51:28 -0700 Subject: [PATCH 44/47] Nits --- .../scala/org/apache/spark/internal/config/package.scala | 1 - .../apache/spark/shuffle/IndexShuffleBlockResolver.scala | 1 - .../spark/storage/BlockManagerDecommissioner.scala | 4 ++-- .../BlockManagerDecommissionIntegrationSuite.scala | 9 ++++----- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6bfe13f287a3e..f2fa99b282cd7 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -436,7 +436,6 @@ package object config { .checkValue(_ > 0, "The maximum number of threads should be positive") .createWithDefault(8) - private[spark] val STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED = ConfigBuilder("spark.storage.decommission.rddBlocks.enabled") .doc("Whether to transfer RDD blocks during block manager decommissioning.") diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 6a77e2c805a06..eddcc9e86cb67 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -76,7 +76,6 @@ private[spark] class IndexShuffleBlockResolver( }.toSet } - /** * Get the shuffle data file. * diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 46f6763a9690c..01f7ed2e4b746 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -115,7 +115,7 @@ private[storage] class BlockManagerDecommissioner( // Shuffles which are either in queue for migrations or migrated private val migratingShuffles = mutable.HashSet[ShuffleBlockInfo]() - // Shuffles which are queued for migration + // Shuffles which are queued for migration & number of retries so far. private[storage] val shufflesToMigrate = new java.util.concurrent.ConcurrentLinkedQueue[(ShuffleBlockInfo, Int)]() @@ -196,7 +196,7 @@ private[storage] class BlockManagerDecommissioner( // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") val localShuffles = bm.migratableResolver.getStoredShuffles() - val newShufflesToMigrate = localShuffles.&~(migratingShuffles).toSeq + val newShufflesToMigrate = localShuffles.diff(migratingShuffles).toSeq shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) migratingShuffles ++= newShufflesToMigrate diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala index 6a315805f6cb9..afcb38bc38836 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala @@ -57,8 +57,8 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS .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. + // 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) @@ -68,7 +68,6 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS 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() @@ -192,8 +191,8 @@ class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with LocalS val blockId = update.blockUpdatedInfo.blockId blockId.isInstanceOf[ShuffleIndexBlockId] }.size - assert(numDataLocs >= 1, s"Expect shuffle data block updates in ${blocksUpdated}") - assert(numIndexLocs >= 1, s"Expect shuffle index block updates in ${blocksUpdated}") + assert(numDataLocs === 1, s"Expect shuffle data block updates in ${blocksUpdated}") + assert(numIndexLocs === 1, s"Expect shuffle index block updates in ${blocksUpdated}") } } From 9d210f53c1907a46a80b2887b47592f5c42b040a Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 16 Jul 2020 15:30:57 -0700 Subject: [PATCH 45/47] Code review feedback, remove regex we don't need, reduce some log levels, add a Since annotaiton. --- .../shuffle/IndexShuffleBlockResolver.scala | 22 ++++++++----------- .../spark/shuffle/MigratableResolver.scala | 5 +++-- .../apache/spark/storage/BlockManager.scala | 4 ++-- .../storage/BlockManagerDecommissioner.scala | 8 +++---- .../BlockManagerDecommissionUnitSuite.scala | 2 +- 5 files changed, 19 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index eddcc9e86cb67..0d0dad6d77ac1 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -47,6 +47,7 @@ import org.apache.spark.util.Utils // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). private[spark] class IndexShuffleBlockResolver( conf: SparkConf, + // var for testing var _blockManager: BlockManager = null) extends ShuffleBlockResolver with Logging with MigratableResolver { @@ -61,19 +62,14 @@ private[spark] class IndexShuffleBlockResolver( /** * 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 + override def getStoredShuffles(): Seq[ShuffleBlockInfo] = { + val allBlocks = blockManager.diskBlockManager.getAllBlocks() + allBlocks.flatMap { + case ShuffleIndexBlockId(shuffleId, mapId, _) => + Some(ShuffleBlockInfo(shuffleId, mapId)) + case _ => + None + } } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala index 891c8282d26de..3851fa6eca19b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.serializer.SerializerManager @@ -28,11 +28,12 @@ import org.apache.spark.storage.BlockId * An experimental trait to allow Spark to migrate shuffle blocks. */ @Experimental +@Since("3.1.0") trait MigratableResolver { /** * Get the shuffle ids that are stored locally. Used for block migrations. */ - def getStoredShuffles(): Set[ShuffleBlockInfo] + def getStoredShuffles(): Seq[ShuffleBlockInfo] /** * Write a provided shuffle block as a stream. Used for block migrations. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 2e8d6696f2603..47af854b6e8ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -668,7 +668,7 @@ private[spark] class BlockManager( } if (blockId.isShuffle) { - logInfo(s"Putting shuffle block ${blockId}") + logDebug(s"Putting shuffle block ${blockId}") try { return migratableResolver.putShuffleBlockAsStream(blockId, serializerManager) } catch { @@ -677,7 +677,7 @@ private[spark] class BlockManager( s"resolver ${shuffleManager.shuffleBlockResolver}") } } - logInfo(s"Putting regular block ${blockId}") + logDebug(s"Putting regular block ${blockId}") // All other blocks val (_, tmpFile) = diskBlockManager.createTempLocalBlock() val channel = new CountingWritableChannel( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 01f7ed2e4b746..1cc7ef6a25f92 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -35,8 +35,8 @@ import org.apache.spark.util.ThreadUtils * It creates a Thread to retry offloading all RDD cache and Shuffle blocks */ private[storage] class BlockManagerDecommissioner( - conf: SparkConf, - bm: BlockManager) extends Logging { + conf: SparkConf, + bm: BlockManager) extends Logging { private val maxReplicationFailuresForDecommission = conf.get(config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK) @@ -195,7 +195,7 @@ private[storage] class BlockManagerDecommissioner( private[storage] def refreshOffloadingShuffleBlocks(): Unit = { // Update the queue of shuffles to be migrated logInfo("Offloading shuffle blocks") - val localShuffles = bm.migratableResolver.getStoredShuffles() + val localShuffles = bm.migratableResolver.getStoredShuffles().toSet val newShufflesToMigrate = localShuffles.diff(migratingShuffles).toSeq shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) migratingShuffles ++= newShufflesToMigrate @@ -223,7 +223,7 @@ private[storage] class BlockManagerDecommissioner( private[storage] def stopOffloadingShuffleBlocks(): Unit = { logInfo("Stopping offloading shuffle blocks.") // Stop as gracefully as possible. - migrationPeers.values.foreach{_.running = false} + migrationPeers.values.foreach{ _.running = false } shuffleMigrationPool.shutdown() shuffleMigrationPool.shutdownNow() } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala index e56febfb95462..5ff1ff05cc4e0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionUnitSuite.scala @@ -44,7 +44,7 @@ class BlockManagerDecommissionUnitSuite extends SparkFunSuite with Matchers { ids: Set[(Int, Long, Int)]): Unit = { when(mockMigratableShuffleResolver.getStoredShuffles()) - .thenReturn(ids.map(triple => ShuffleBlockInfo(triple._1, triple._2)).toSet) + .thenReturn(ids.map(triple => ShuffleBlockInfo(triple._1, triple._2)).toSeq) ids.foreach { case (shuffleId: Int, mapId: Long, reduceId: Int) => when(mockMigratableShuffleResolver.getMigrationBlocks(mc.any())) From 2467732b5b537e9a39ef4b684c4998f619dba5b1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 17 Jul 2020 14:09:05 -0700 Subject: [PATCH 46/47] Now that we delegate this to the existing blockmanager logic we don't need an explicit test for it here and the block manager is stubbed out in IndexShuffleBlockResolverSuite --- .../spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 71f4d13cb8175..725a1d90557a2 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -74,7 +74,6 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) - val storedShuffles = resolver.getStoredShuffles() val indexFile = new File(tempDir.getAbsolutePath, idxName) val dataFile = resolver.getDataFile(shuffleId, mapId) @@ -83,7 +82,6 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(dataFile.exists()) assert(dataFile.length() === 30) assert(!dataTmp.exists()) - assert(storedShuffles === Set(ShuffleBlockInfo(1, 2))) val lengths2 = new Array[Long](3) val dataTmp2 = File.createTempFile("shuffle", null, tempDir) From 8494bdd94285c7cc5a41e151da920710be7f4671 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 17 Jul 2020 16:45:08 -0700 Subject: [PATCH 47/47] Disable the R integration tests, they've been broken for awhile. --- .../spark/deploy/k8s/integrationtest/KubernetesSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index bb7126057bf60..ebf71e8cb83e4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -42,7 +42,8 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually + // TODO(SPARK-32354): Fix and re-enable the R tests. + with DepsTestsSuite with DecommissionSuite /* with RTestsSuite */ with Logging with Eventually with Matchers {