From b968e0e2af2dfd27009e954a44e38f1e69fcf12c Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 8 May 2020 13:25:31 -0700 Subject: [PATCH 01/35] LIHADOOP-48527 Enable Shuffle writer to push blocks to remote shuffle services. It also includes the below commits: 1. commit 1d63f190fdd7c6839161a048abb29084be97efba Author: Chandni Singh Date: Thu Mar 12 19:32:31 2020 -0700 LIHADOOP-51112 Create connections outside netty event loop RB=1977902 BUG=LIHADOOP-51112 G=superfriends-reviewers R=mshen 2. commit 16800055ad9188850014726f4b3bf69abb354aec Author: Chandni Singh Date: Mon Mar 16 17:31:20 2020 -0700 LIHADOOP-51891 prepareBlockPushRequests in ShuffleWriter uses parameters that are initialized later RB=2010539 BUG=LIHADOOP-51891 G=superfriends-reviewers R=mshen --- .../sort/BypassMergeSortShuffleWriter.java | 5 +- .../shuffle/sort/UnsafeShuffleWriter.java | 7 +- .../org/apache/spark/executor/Executor.scala | 3 +- .../spark/internal/config/package.scala | 25 ++ .../spark/shuffle/ShuffleWriteProcessor.scala | 15 +- .../apache/spark/shuffle/ShuffleWriter.scala | 398 +++++++++++++++++- .../shuffle/sort/SortShuffleWriter.scala | 6 +- .../BypassMergeSortShuffleWriterSuite.scala | 51 +++ 8 files changed, 500 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 256789b8c7827..3dbee1b13d287 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -31,7 +31,6 @@ import scala.Tuple2; import scala.collection.Iterator; -import com.google.common.annotations.VisibleForTesting; import com.google.common.io.Closeables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -178,8 +177,8 @@ public void write(Iterator> records) throws IOException { } } - @VisibleForTesting - long[] getPartitionLengths() { + @Override + public long[] getPartitionLengths() { return partitionLengths; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 79e38a824fea4..e8f94ba8ffeee 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -88,6 +88,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; + @Nullable private long[] partitionLengths; private long peakMemoryUsedBytes = 0; /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ @@ -219,7 +220,6 @@ void closeAndWriteOutput() throws IOException { serOutputStream = null; final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; - final long[] partitionLengths; try { partitionLengths = mergeSpills(spills); } finally { @@ -543,4 +543,9 @@ public void close() throws IOException { channel.close(); } } + + @Override + public long[] getPartitionLengths() { + return partitionLengths; + } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e7f1b8f3cf17a..6c251b54ef474 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.{FetchFailedException, ShuffleWriter} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,6 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } + ShuffleWriter.stop threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly 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 6639f20a068d4..43f5ca9d7dc54 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 @@ -2023,4 +2023,29 @@ package object config { .version("3.1.0") .doubleConf .createWithDefault(5) + + private[spark] val PUSH_BASED_SHUFFLE_PUSHER_THREADS = + ConfigBuilder("spark.shuffle.push.based.pusherThreads") + .doc("Specify the number of threads in the block pusher pool. These threads assist " + + "in creating connections and pushing blocks to remote shuffle services when push based " + + "shuffle is enabled. By default, the threadpool size is equal to the number of cores") + .intConf + .createOptional + + private[spark] val PUSH_BASED_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = + ConfigBuilder("spark.shuffle.push.based.maxBlockSizeToPush") + .doc("The max size of an individual block to push to the remote shuffle services when push " + + "based shuffle is enabled. Blocks larger than this threshold are not pushed.") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("800k") + + private[spark] val PUSH_BASED_SHUFFLE_MAX_BLOCK_BATCH_SIZE = + ConfigBuilder("spark.shuffle.push.based.maxBlockBatchSize") + .doc("The max size of a batch of shuffle blocks to be grouped into a single push request " + + "when push based shuffle is enabled.") + .bytesConf(ByteUnit.MiB) + // 2m is also the default value for TransportConf#memoryMapBytes. + // Having this default to 2m will very likely make each batch of block loaded in memory with + // memory mapping, which has higher overhead with small MB sized chunk of data. + .createWithDefaultString("3m") } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 1429144c6f6e2..5fb50719efefc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -21,6 +21,7 @@ import org.apache.spark.{Partition, ShuffleDependency, SparkEnv, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.MapStatus +import org.apache.spark.util.Utils /** * The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor @@ -57,7 +58,19 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { createMetricsReporter(context)) writer.write( rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) - writer.stop(success = true).get + val mapStatus = writer.stop(success = true) + if (mapStatus.isDefined) { + // Initiate shuffle push process if push based shuffle is enabled + // The map task only takes care of converting the shuffle data file into multiple + // block push requests and pushing the first batch of blocks to the remote shuffle + // service. The remaining block pushes is handled by netty client threads. + if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf)) { + writer.initiateBlockPush( + manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], + writer.getPartitionLengths(), dep, partition.index, mapId, SparkEnv.get.conf) + } + } + mapStatus.get } catch { case e: Exception => try { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 4cc4ef5f1886e..11d27512f4b86 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -17,18 +17,410 @@ package org.apache.spark.shuffle -import java.io.IOException +import java.io.{File, IOException} +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, + BlockStoreClient} +import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.ShuffleWriter._ +import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.util.{ThreadUtils, Utils} /** - * Obtained inside a map task to write out records to the shuffle system. + * Obtained inside a map task to write out records to the shuffle system, and optionally + * initiate the block push process to remote shuffle services if push based shuffle is enabled. */ -private[spark] abstract class ShuffleWriter[K, V] { +private[spark] abstract class ShuffleWriter[K, V] extends Logging { + private[this] var maxBytesInFlight = 0L + private[this] var maxReqsInFlight = 0 + private[this] var maxBlocksInFlightPerAddress = 0 + private[this] var bytesInFlight = 0L + private[this] var reqsInFlight = 0 + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() + private[this] val pushRequests = new Queue[PushRequest] + /** Write a sequence of records to this task's output */ @throws[IOException] def write(records: Iterator[Product2[K, V]]): Unit /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] + + def getPartitionLengths(): Array[Long] + + /** + * Initiate the block push process. This will be invoked after the shuffle writer + * finishes writing the shuffle file if push based shuffle is enabled. + * + * @param resolver block resolver used to locate mapper generated shuffle file + * @param partitionLengths array of shuffle block size so we can tell shuffle block + * boundaries within the shuffle file + * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle + * services to push local shuffle blocks + * @param partitionId map index of the shuffle map task + * @param mapId mapId of the shuffle map task + * @param conf spark configuration + */ + def initiateBlockPush( + resolver: IndexShuffleBlockResolver, + partitionLengths: Array[Long], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + mapId: Long, + conf: SparkConf): Unit = { + val numPartitions = dep.partitioner.numPartitions + val dataFile = resolver.getDataFile(dep.shuffleId, mapId) + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") + + val maxBlockSizeToPush = conf.get(PUSH_BASED_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + val maxBlockBatchSize = conf.get(PUSH_BASED_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 + val mergerLocs = dep.getMergerLocs.map(loc => + BlockManagerId("", loc.host, loc.port)) + + maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + maxBlocksInFlightPerAddress = conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) + + val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, + partitionLengths, mergerLocs, transportConf, maxBlockSizeToPush, maxBlockBatchSize) + // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same + // time won't be pushing the same ranges of shuffle partitions. + pushRequests ++= Utils.randomize(requests) + + val shuffleClient = SparkEnv.get.blockManager.blockStoreClient + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.execute(() => pushUpToMax(shuffleClient)) + } + } + + /** + * Since multiple netty client threads could potentially be calling pushUpToMax for the same + * mapper, we synchronize access to this method so that only one thread can push blocks for + * a given mapper. This helps to simplify access to the shared states. The down side of this + * is that we could unnecessarily block other mappers' block pushes if all netty client threads + * are occupied by block pushes from the same mapper. + * + * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles + * the data transfer between shuffle client/server. + */ + private def pushUpToMax(shuffleClient: BlockStoreClient): Unit = synchronized { + // Process any outstanding deferred push requests if possible. + if (deferredPushRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredPushRequests) { + while (isRemoteBlockPushable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred push request for $remoteAddress with " + + s"${request.blocks.length} blocks") + sendRequest(request, shuffleClient) + if (defReqQueue.isEmpty) { + deferredPushRequests -= remoteAddress + } + } + } + } + + // Process any regular push requests if possible. + while (isRemoteBlockPushable(pushRequests)) { + val request = pushRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") + val defReqQueue = deferredPushRequests.getOrElse(remoteAddress, new Queue[PushRequest]()) + defReqQueue.enqueue(request) + deferredPushRequests(remoteAddress) = defReqQueue + } else { + sendRequest(request, shuffleClient) + } + } + + def isRemoteBlockPushable(pushReqQueue: Queue[PushRequest]): Boolean = { + pushReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + pushReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new push request will exceed the max no. of blocks being pushed to a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: PushRequest): Boolean = { + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + + request.blocks.size > maxBlocksInFlightPerAddress + } + } + + /** + * Push blocks to remote shuffle server. The callback listener will invoke #pushUpToMax again + * to trigger pushing the next batch of blocks once some block transfer is done in the current + * batch. This way, we decouple the map task from the block push process, since it is netty + * client thread instead of task execution thread which takes care of majority of the block + * pushes. + */ + private def sendRequest(request: PushRequest, shuffleClient: BlockStoreClient): Unit = { + bytesInFlight = bytesInFlight + request.size + reqsInFlight = reqsInFlight + 1 + numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( + request.address, 0) + request.blocks.length + + val sizeMap = request.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val address = request.address + val blockIds = request.blocks.map(_._1.toString) + val remainingBlocks = new HashSet[String]() ++= blockIds + + val blockPushListener = new BlockFetchingListener { + + // Initiating a connection and pushing blocks to a remote shuffle service is always handled by + // the block-push-threads. We don't initiate the connection creation in the + // blockPushListener callbacks which are invoked by the netty eventloop because: + // 1. TrasportClient.createConnection(...) blocks for connection to be established and it's + // recommended to avoid any blocking operations in the eventloop; + // 2. The actual connection creation is a task that gets added to the task queue of another + // eventloop which could have eventloops eventually blocking each other. + // Once the blockPushListener is notified of the block push success or failure, we + // just delegate it to block-push-threads. + def handleResult(result: PushResult): Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.execute(new Runnable { + override def run(): Unit = { + if (updateStateAndCheckIfPushMore( + sizeMap(result.blockId), address, remainingBlocks, result)) { + pushUpToMax(SparkEnv.get.blockManager.blockStoreClient) + } + } + }) + } + } + + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + logTrace(s"Push for block $blockId to $address successful.") + handleResult(PushResult(blockId, null)) + } + + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + logWarning(s"Pushing block $blockId to $address failed.", exception) + handleResult(PushResult(blockId, exception)) + } + } + shuffleClient.pushBlocks(address.host, address.port, blockIds.toArray, + sliceReqBufferIntoBlockBuffers(request.reqBuffer, request.blocks.map(_._2)), + blockPushListener) + } + + /** + * Given the ManagedBuffer representing all the continuous blocks inside the shuffle data file + * for a PushRequest and an array of individual block sizes, load the buffer from disk into + * memory and slice it into multiple smaller buffers representing each block. + * + * With nio ByteBuffer, the individual block buffers share data with the initial in memory + * buffer loaded from disk. Thus only one copy of the block data is kept in memory. + * @param reqBuffer A {{FileSegmentManagedBuffer}} representing all the continuous blocks in + * the shuffle data file for a PushRequest + * @param blockSizes Array of block sizes + * @return Array of in memory buffer for each individual block + */ + private def sliceReqBufferIntoBlockBuffers( + reqBuffer: ManagedBuffer, + blockSizes: Seq[Long]): Array[ManagedBuffer] = { + if (blockSizes.size == 1) { + Seq(reqBuffer).toArray + } else { + val inMemoryBuffer = reqBuffer.nioByteBuffer() + val blockOffsets = new Array[Long](blockSizes.size) + var offset = 0L + for (index <- blockSizes.indices) { + blockOffsets(index) = offset + offset += blockSizes(index) + } + blockOffsets.zip(blockSizes).map { + case (offset, size) => + new NioManagedBuffer(inMemoryBuffer.duplicate() + .position(offset.toInt) + .limit((offset + size).toInt).asInstanceOf[ByteBuffer].slice()) + }.toArray + } + } + + /** + * Updates the stats and based on the previous push result decides whether to push more blocks + * or stop. + * + * @param bytesPushed number of bytes pushed. + * @param address address of the remote service + * @param remainingBlocks remaining blocks + * @param pushResult result of the last push + * @return true if more blocks should be pushed; false otherwise. + */ + private def updateStateAndCheckIfPushMore( + bytesPushed: Long, + address: BlockManagerId, + remainingBlocks: HashSet[String], + pushResult: PushResult): Boolean = synchronized { + remainingBlocks -= pushResult.blockId + + bytesInFlight = bytesInFlight - bytesPushed + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + if (remainingBlocks.isEmpty) { + reqsInFlight = reqsInFlight - 1 + } + if (pushResult.failure != null && pushResult.failure.getCause != null && + pushResult.failure.getCause.getMessage != null && + pushResult.failure.getCause.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) { + false + } else { + remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) + } + } + + /** + * Convert the shuffle data file of the current mapper into a list of PushRequest. Basically, + * continuous blocks in the shuffle file are grouped into a single request to allow more + * efficient read of the block data. Each mapper for a given shuffle will receive the same + * list of BlockManagerIds as the target location to push the blocks to. All mappers in the + * same shuffle will map shuffle partition ranges to individual target locations in a consistent + * manner to make sure each target location receives shuffle blocks belonging to the same set + * of partition ranges. 0-length blocks and blocks that are large enough will be skipped. + * + * @param numPartitions Number of shuffle partitions in the shuffle file + * @param partitionId map index of the current mapper + * @param shuffleId ShuffleId of current shuffle + * @param dataFile Shuffle data file + * @param partitionLengths array of sizes of blocks in the shuffle data file + * @param mergerLocs Target locations to push blocks to + * @param transportConf TransportConf used to create FileSegmentManagedBuffer + * @param maxBlockSizeToPush Max size of individual blocks that will be pushed. Blocks larger + * than this threshold will be skipped. + * @param maxBlockBatchSize Max size of a batch of shuffle blocks to be grouped into a single + * request + * @return List of the PushRequest, randomly shuffled. + */ + private def prepareBlockPushRequests( + numPartitions: Int, + partitionId: Int, + shuffleId: Int, + dataFile: File, + partitionLengths: Array[Long], + mergerLocs: Seq[BlockManagerId], + transportConf: TransportConf, + maxBlockSizeToPush: Long, + maxBlockBatchSize: Long): Seq[PushRequest] = { + var offset = 0L + var currentReqSize = 0L + var currentReqOffset = 0L + var currentMergerId = 0 + val numMergers = mergerLocs.length + val requests = new ArrayBuffer[PushRequest] + var blocks = new ArrayBuffer[(BlockId, Long)] + for (reduceId <- 0 until numPartitions) { + val blockSize = partitionLengths(reduceId) + logDebug(s"Block ${ShuffleBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") + // Skip 0-length blocks and blocks that are large enough + if (blockSize > 0) { + val mergerId = math.min(math.floor(reduceId * 1.0 / numPartitions * numMergers), + numMergers - 1).asInstanceOf[Int] + // Start a new PushRequest if the current request goes beyond the max batch size, + // or the number of blocks in the current request goes beyond the limit per destination, + // or the next block push location is for a different shuffle service, or the next block + // exceeds the max block size to push limit. This guarantees that each PushReqeust + // represents continuous blocks in the shuffle file to be pushed to the same shuffle + // service, and does not go beyond existing limitations. + if (currentReqSize + blockSize <= maxBlockBatchSize && + blocks.size < maxBlocksInFlightPerAddress && + mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { + // Add current block to current batch + currentReqSize += blockSize + } else { + if (blocks.nonEmpty) { + // Convert the previous batch into a PushRequest + requests += PushRequest(mergerLocs(currentMergerId), blocks, + new FileSegmentManagedBuffer(transportConf, dataFile, + currentReqOffset, currentReqSize)) + } + // Start a new batch + currentReqSize = 0 + // Set currentReqffset to -1 so we are able to distinguish between the initial value + // of currentReqOffset and when we are about to start a new batch + currentReqOffset = -1 + currentMergerId = mergerId + blocks = new ArrayBuffer[(BlockId, Long)] + } + // Skip blocks exceeding the size limit for push + if (blockSize <= maxBlockSizeToPush) { + blocks += ((ShuffleBlockId(shuffleId, partitionId, reduceId), blockSize)) + // Only update currentReqOffset if the current block is the first in the request + if (currentReqOffset == -1) { + currentReqOffset = offset + } + if (currentReqSize == 0) { + currentReqSize += blockSize + } + } + } + offset = offset + blockSize + } + // Add in the final request + if (blocks.nonEmpty) { + requests += PushRequest(mergerLocs(currentMergerId), blocks, + new FileSegmentManagedBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + } + requests + } +} + +private[spark] object ShuffleWriter { + + /** + * A request to push blocks to a remote shuffle service + * @param address remote shuffle service location to push blocks to + * @param blocks list of block IDs and their sizes + * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous + * blocks represented in this request + */ + case class PushRequest( + address: BlockManagerId, + blocks: Seq[(BlockId, Long)], + reqBuffer: ManagedBuffer) { + val size = blocks.map(_._2).sum + } + + /** + * Result of the block push. + * @param blockId blockId + * @param failure exception if the push was unsuccessful; null otherwise; + */ + private case class PushResult( + blockId: String, + failure: Throwable + ) + + private val BLOCK_PUSHER_POOL: ExecutorService = { + val conf = SparkEnv.get.conf + if (Utils.isPushBasedShuffleEnabled(conf)) { + val numThreads = conf.get(PUSH_BASED_SHUFFLE_PUSHER_THREADS) + .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) + ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") + } else { + null + } + } + + /** + * Stop the shuffle writer pool if it isn't null. + */ + private[spark] def stop : Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.shutdown() + } + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 83ebe3e12946c..af8d1e2fff413 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -45,6 +45,8 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null + private var partitionLengths: Array[Long] = _ + private val writeMetrics = context.taskMetrics().shuffleWriteMetrics /** Write a bunch of records to this task's output */ @@ -67,7 +69,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - val partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths + partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } @@ -93,6 +95,8 @@ private[spark] class SortShuffleWriter[K, V, C]( } } } + + override def getPartitionLengths(): Array[Long] = partitionLengths } private[spark] object SortShuffleWriter { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index f8474022867f4..01a1ddee719fd 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.shuffle.sort import java.io.File import java.util.UUID +import java.util.concurrent.{Semaphore, TimeUnit} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -27,11 +28,14 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents @@ -71,6 +75,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) + when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) @@ -236,4 +242,49 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } + + // TODO: Move this to ShuffleWriterSuite. It passes in the IDE but fails consistently with mvn. + ignore("test block push") { + def records: Iterator[(Int, Int)] = Iterator((1, 1), (2, 2), (3, 3)) + + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + shuffleHandle, + 0, // MapId + conf, + taskContext.taskMetrics().shuffleWriteMetrics, + shuffleExecutorComponents + ) + conf.set("spark.shuffle.push.based.enabled", "true") + conf.set("spark.shuffle.service.enabled", "true") + val mockEnv = mock(classOf[SparkEnv]) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.blockManager).thenReturn(blockManager) + val shuffleClient = mock(classOf[BlockStoreClient]) + when(blockManager.blockStoreClient).thenReturn(shuffleClient) + SparkEnv.set(mockEnv) + + val sem = new Semaphore(0) + var blockIds : Array[String] = null + when(shuffleClient.pushBlocks(any(), anyInt(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + blockIds = invocation.getArguments()(2).asInstanceOf[Array[String]] + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = + invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + (blockIds, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + sem.release() + }) + + writer.write(records) + writer.initiateBlockPush(blockResolver, writer.getPartitionLengths(), dependency, 0, 0, conf) + + sem.tryAcquire(1, 500, TimeUnit.MILLISECONDS) + for (i <- 1 to 3) { + assert(blockIds(i - 1) === ShuffleBlockId(0, 0, i).toString) + } + writer.stop(true) + } } From 510f5049860dd246df2b0614253c6a1032040931 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 14 May 2020 17:36:09 -0700 Subject: [PATCH 02/35] LIHADOOP-53496 Not logging all block push exceptions on the client RB=2104829 BUG=LIHADOOP-53496 G=spark-reviewers R=yezhou,mshen A=mshen --- .../apache/spark/shuffle/ShuffleWriter.scala | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 11d27512f4b86..9b391c3a2161e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -209,7 +209,16 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - logWarning(s"Pushing block $blockId to $address failed.", exception) + if ((exception.getMessage != null && + exception.getMessage.contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)) || + (exception.getCause != null && exception.getCause.getMessage != null && + exception.getCause.getMessage.contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX))) { + logTrace(s"Pushing block $blockId to $address failed.", exception) + } else { + logWarning(s"Pushing block $blockId to $address failed.", exception) + } handleResult(PushResult(blockId, exception)) } } @@ -274,9 +283,12 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { if (remainingBlocks.isEmpty) { reqsInFlight = reqsInFlight - 1 } - if (pushResult.failure != null && pushResult.failure.getCause != null && - pushResult.failure.getCause.getMessage != null && - pushResult.failure.getCause.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) { + if (pushResult.failure != null && + ((pushResult.failure.getMessage != null && + pushResult.failure.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) || + (pushResult.failure.getCause != null && pushResult.failure.getCause.getMessage != null && + pushResult.failure.getCause.getMessage.contains( + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { false } else { remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) From 2d9d27d77ab61838332634c596404e487f9a5496 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Thu, 25 Jun 2020 10:50:15 -0700 Subject: [PATCH 03/35] LIHADOOP-54374 Separate the configurations for connection creation timeout and connection idle timeout RB=2165761 BUG=LIHADOOP-54374 G=spark-reviewers R=mshen,chsingh A=mshen,chsingh --- .../spark/network/client/TransportClientFactory.java | 7 ++++--- .../org/apache/spark/network/util/TransportConf.java | 11 ++++++++++- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 24c436a504fa8..43408d43e577e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -254,7 +254,7 @@ TransportClient createClient(InetSocketAddress address) // Disable Nagle's Algorithm since we don't want packets to wait .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionCreationTimeoutMs()) .option(ChannelOption.ALLOCATOR, pooledAllocator); if (conf.receiveBuf() > 0) { @@ -280,9 +280,10 @@ public void initChannel(SocketChannel ch) { // Connect to the remote server long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); - if (!cf.await(conf.connectionTimeoutMs())) { + if (!cf.await(conf.connectionCreationTimeoutMs())) { throw new IOException( - String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + String.format("Connecting to %s timed out (%s ms)", + address, conf.connectionCreationTimeoutMs())); } else if (cf.cause() != null) { throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index fd287b022618b..49f4c2024312f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -31,6 +31,7 @@ public class TransportConf { private final String SPARK_NETWORK_IO_MODE_KEY; private final String SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY; private final String SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY; + private final String SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY; private final String SPARK_NETWORK_IO_BACKLOG_KEY; private final String SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY; private final String SPARK_NETWORK_IO_SERVERTHREADS_KEY; @@ -54,6 +55,7 @@ public TransportConf(String module, ConfigProvider conf) { SPARK_NETWORK_IO_MODE_KEY = getConfKey("io.mode"); SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY = getConfKey("io.preferDirectBufs"); SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY = getConfKey("io.connectionTimeout"); + SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY = getConfKey("io.connectionCreationTimeout"); SPARK_NETWORK_IO_BACKLOG_KEY = getConfKey("io.backLog"); SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY = getConfKey("io.numConnectionsPerPeer"); SPARK_NETWORK_IO_SERVERTHREADS_KEY = getConfKey("io.serverThreads"); @@ -94,7 +96,7 @@ public boolean preferDirectBufs() { return conf.getBoolean(SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); } - /** Connect timeout in milliseconds. Default 120 secs. */ + /** Connect idle timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( conf.get("spark.network.timeout", "120s")); @@ -103,6 +105,13 @@ public int connectionTimeoutMs() { return (int) defaultTimeoutMs; } + /** Connect creation timeout in milliseconds. Default 30 secs. */ + public int connectionCreationTimeoutMs() { + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get(SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY, "30s")) * 1000; + return (int) defaultTimeoutMs; + } + /** Number of concurrent connections between two nodes for fetching data. */ public int numConnectionsPerPeer() { return conf.getInt(SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY, 1); From 3d742775d98334769615d7e0a7583c07a21f3c7e Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 23 Jun 2020 23:10:07 -0700 Subject: [PATCH 04/35] LIHADOOP-54370 Not to retry on certain exceptions when pushing blocks RB=2166258 BUG=LIHADOOP-54370 G=spark-reviewers R=mshen,yezhou A=mshen --- .../spark/shuffle/ShuffleWriteProcessor.scala | 2 +- .../apache/spark/shuffle/ShuffleWriter.scala | 98 +++++++--- .../spark/shuffle/ShuffleWriterSuite.scala | 173 ++++++++++++++++++ .../BypassMergeSortShuffleWriterSuite.scala | 51 ------ 4 files changed, 243 insertions(+), 81 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 5fb50719efefc..92b1756f6a01d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -64,7 +64,7 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { // The map task only takes care of converting the shuffle data file into multiple // block push requests and pushing the first batch of blocks to the remote shuffle // service. The remaining block pushes is handled by netty client threads. - if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf)) { + if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { writer.initiateBlockPush( manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], writer.getPartitionLengths(), dep, partition.index, mapId, SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 9b391c3a2161e..2000ca88101aa 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -18,25 +18,27 @@ package org.apache.spark.shuffle import java.io.{File, IOException} +import java.net.ConnectException import java.nio.ByteBuffer import java.util.concurrent.ExecutorService import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} - import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} -import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.{Logging, config} import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, - BlockStoreClient} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.ShuffleWriter._ import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.{ThreadUtils, Utils} +import scala.collection.mutable + /** * Obtained inside a map task to write out records to the shuffle system, and optionally * initiate the block push process to remote shuffle services if push based shuffle is enabled. @@ -50,6 +52,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() private[this] val pushRequests = new Queue[PushRequest] + private[this] val errorHandler = createErrorHandler() + private[this] val unreachableBlockMgrs = new HashSet[BlockManagerId]() /** Write a sequence of records to this task's output */ @throws[IOException] @@ -60,6 +64,24 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { def getPartitionLengths(): Array[Long] + /** + * VisbleForTesting + */ + private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { + new BlockPushErrorHandler() { + override def shouldRetryError(t: Throwable): Boolean = { + // If the block is too late, there is no need to retry it + if ((t.getMessage != null && + t.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) || + (t.getCause != null && t.getCause.getMessage != null && + t.getCause.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) { + return false + } + true + } + } + } + /** * Initiate the block push process. This will be invoked after the shuffle writer * finishes writing the shuffle file if push based shuffle is enabled. @@ -100,8 +122,18 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { pushRequests ++= Utils.randomize(requests) val shuffleClient = SparkEnv.get.blockManager.blockStoreClient + submitTask(() => { + pushUpToMax(shuffleClient) + }) + } + + /** + * Triggers the push. It's a separate method for testing. + * Visible for testing + */ + protected def submitTask(task: Runnable): Unit = { if (BLOCK_PUSHER_POOL != null) { - BLOCK_PUSHER_POOL.execute(() => pushUpToMax(shuffleClient)) + BLOCK_PUSHER_POOL.execute(task) } } @@ -191,16 +223,12 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // Once the blockPushListener is notified of the block push success or failure, we // just delegate it to block-push-threads. def handleResult(result: PushResult): Unit = { - if (BLOCK_PUSHER_POOL != null) { - BLOCK_PUSHER_POOL.execute(new Runnable { - override def run(): Unit = { - if (updateStateAndCheckIfPushMore( - sizeMap(result.blockId), address, remainingBlocks, result)) { - pushUpToMax(SparkEnv.get.blockManager.blockStoreClient) - } - } - }) - } + submitTask(() => { + if (updateStateAndCheckIfPushMore( + sizeMap(result.blockId), address, remainingBlocks, result)) { + pushUpToMax(SparkEnv.get.blockManager.blockStoreClient) + } + }) } override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { @@ -209,12 +237,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - if ((exception.getMessage != null && - exception.getMessage.contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)) || - (exception.getCause != null && exception.getCause.getMessage != null && - exception.getCause.getMessage.contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX))) { + // check the message or it's cause to see it needs to be logged. + if (!errorHandler.shouldLogError(exception)) { logTrace(s"Pushing block $blockId to $address failed.", exception) } else { logWarning(s"Pushing block $blockId to $address failed.", exception) @@ -238,8 +262,10 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { * the shuffle data file for a PushRequest * @param blockSizes Array of block sizes * @return Array of in memory buffer for each individual block + * + * VisibleForTesting */ - private def sliceReqBufferIntoBlockBuffers( + protected def sliceReqBufferIntoBlockBuffers( reqBuffer: ManagedBuffer, blockSizes: Seq[Long]): Array[ManagedBuffer] = { if (blockSizes.size == 1) { @@ -283,13 +309,27 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { if (remainingBlocks.isEmpty) { reqsInFlight = reqsInFlight - 1 } - if (pushResult.failure != null && - ((pushResult.failure.getMessage != null && - pushResult.failure.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) || - (pushResult.failure.getCause != null && pushResult.failure.getCause.getMessage != null && - pushResult.failure.getCause.getMessage.contains( - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { - false + if (pushResult.failure != null && pushResult.failure.getCause != null && + pushResult.failure.getCause.isInstanceOf[ConnectException]) { + // Remove all the blocks for this address just once because removing from pushRequests + // is expensive. If there is a ConnectException for the first block, all the subsequent + // blocks to that address will fail, so should avoid removing multiple times. + if (!unreachableBlockMgrs.contains(address)) { + var removed = 0 + unreachableBlockMgrs.add(address) + removed += pushRequests.dequeueAll(req => req.address == address).length + val droppedReq = deferredPushRequests.remove(address) + if (droppedReq.isDefined) { + removed += droppedReq.get.length + } + logWarning(s"Received a ConnectException from $address. " + + s"Dropping push of $removed blocks and " + + s"not pushing any more blocks to this address.") + } + } + if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { + logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") + return false } else { remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala new file mode 100644 index 0000000000000..8fbf76334efa0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala @@ -0,0 +1,173 @@ +/* + * 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 java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService + +import scala.collection.mutable.ArrayBuffer + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage._ + +class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ + @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ + + private val conf: SparkConf = new SparkConf(loadDefaults = false) + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + when(dependency.partitioner).thenReturn(new HashPartitioner(8)) + when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) + conf.set("spark.shuffle.push.based.enabled", "true") + conf.set("spark.shuffle.service.enabled", "true") + // Set the env because the shuffler writer gets the shuffle client instance from the env. + val mockEnv = mock(classOf[SparkEnv]) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.blockManager).thenReturn(blockManager) + SparkEnv.set(mockEnv) + when(blockManager.blockStoreClient).thenReturn(shuffleClient) + } + + test("test block push") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val allBlocks = new ArrayBuffer[String] + + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + allBlocks ++= pushedBlocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + (pushedBlocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(allBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("error retries") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val errorHandler = testWriter.createErrorHandler() + assert( + !errorHandler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) + assert( + errorHandler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + assert (errorHandler.shouldRetryError(new Throwable())) + } + + test("error logging") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val errorHandler = testWriter.createErrorHandler() + assert( + !errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + assert( + !errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + assert(errorHandler.shouldLogError(new Throwable())) + } + + test("connect exceptions removes all the push requests for that host") { + when(dependency.getMergerLocs).thenReturn( + Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + val executorService = mock(classOf[ExecutorService]) + when(executorService.submit(any[Runnable]())).thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + } + }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val allBlocks = new ArrayBuffer[String] + + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + allBlocks ++= pushedBlocks + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + pushedBlocks.foreach(blockId => { + blockFetchListener.onBlockFetchFailure( + blockId, new RuntimeException(new ConnectException())) + }) + }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(2)) + .pushBlocks(any(), any(), any(), any(), any()) + // 2 blocks for each merger locations + assert(allBlocks.length == 4) + } + + private class TestShuffleWriter( + private val numPartitions: Int) extends ShuffleWriter[Int, Int] { + + override protected def submitTask(task: Runnable): Unit = { + // Making this synchronous for testing + task.run() + } + + /** Write a sequence of records to this task's output */ + override def write(records: Iterator[Product2[Int, Int]]): Unit = {} + + /** Close this writer, passing along whether the map completed */ + override def stop(success: Boolean): Option[MapStatus] = { + Option.empty + } + + override def getPartitionLengths(): Array[Long] = { + Array.fill(numPartitions) { + 2 + } + } + + override protected def sliceReqBufferIntoBlockBuffers( + reqBuffer: ManagedBuffer, blockSizes: Seq[Long]) = { + Array.fill(blockSizes.length) { + new NioManagedBuffer(ByteBuffer.wrap(Array[Byte](2))) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 01a1ddee719fd..f8474022867f4 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.shuffle.sort import java.io.File import java.util.UUID -import java.util.concurrent.{Semaphore, TimeUnit} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -28,14 +27,11 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents @@ -75,8 +71,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) - when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) - when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) @@ -242,49 +236,4 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } - - // TODO: Move this to ShuffleWriterSuite. It passes in the IDE but fails consistently with mvn. - ignore("test block push") { - def records: Iterator[(Int, Int)] = Iterator((1, 1), (2, 2), (3, 3)) - - val writer = new BypassMergeSortShuffleWriter[Int, Int]( - blockManager, - shuffleHandle, - 0, // MapId - conf, - taskContext.taskMetrics().shuffleWriteMetrics, - shuffleExecutorComponents - ) - conf.set("spark.shuffle.push.based.enabled", "true") - conf.set("spark.shuffle.service.enabled", "true") - val mockEnv = mock(classOf[SparkEnv]) - when(mockEnv.conf).thenReturn(conf) - when(mockEnv.blockManager).thenReturn(blockManager) - val shuffleClient = mock(classOf[BlockStoreClient]) - when(blockManager.blockStoreClient).thenReturn(shuffleClient) - SparkEnv.set(mockEnv) - - val sem = new Semaphore(0) - var blockIds : Array[String] = null - when(shuffleClient.pushBlocks(any(), anyInt(), any(), any(), any())) - .thenAnswer((invocation: InvocationOnMock) => { - blockIds = invocation.getArguments()(2).asInstanceOf[Array[String]] - val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] - val blockFetchListener = - invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - (blockIds, managedBuffers).zipped.foreach((blockId, buffer) => { - blockFetchListener.onBlockFetchSuccess(blockId, buffer) - }) - sem.release() - }) - - writer.write(records) - writer.initiateBlockPush(blockResolver, writer.getPartitionLengths(), dependency, 0, 0, conf) - - sem.tryAcquire(1, 500, TimeUnit.MILLISECONDS) - for (i <- 1 to 3) { - assert(blockIds(i - 1) === ShuffleBlockId(0, 0, i).toString) - } - writer.stop(true) - } } From 02cff3b5a636b3ef42df3d7b2b51366c2d066a8e Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 5 Nov 2020 21:56:02 -0800 Subject: [PATCH 05/35] Made the code compile, added more tests, introduced ShuffleBlockPushId --- .../spark/network/util/TransportConf.java | 2 +- .../spark/internal/config/package.scala | 15 +- .../apache/spark/shuffle/ShuffleWriter.scala | 79 ++++---- .../org/apache/spark/storage/BlockId.scala | 8 + .../spark/shuffle/ShuffleWriterSuite.scala | 180 +++++++++++++----- 5 files changed, 193 insertions(+), 91 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 49f4c2024312f..68a75a616a1ff 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -96,7 +96,7 @@ public boolean preferDirectBufs() { return conf.getBoolean(SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); } - /** Connect idle timeout in milliseconds. Default 120 secs. */ + /** Connection idle timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( conf.get("spark.network.timeout", "120s")); 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 43f5ca9d7dc54..2081382c9407d 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 @@ -2024,25 +2024,28 @@ package object config { .doubleConf .createWithDefault(5) - private[spark] val PUSH_BASED_SHUFFLE_PUSHER_THREADS = - ConfigBuilder("spark.shuffle.push.based.pusherThreads") + private[spark] val PUSH_SHUFFLE_NUM_PUSH_THREADS = + ConfigBuilder("spark.shuffle.push.numPushThreads") .doc("Specify the number of threads in the block pusher pool. These threads assist " + "in creating connections and pushing blocks to remote shuffle services when push based " + "shuffle is enabled. By default, the threadpool size is equal to the number of cores") + .version("3.1.0") .intConf .createOptional - private[spark] val PUSH_BASED_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = - ConfigBuilder("spark.shuffle.push.based.maxBlockSizeToPush") + private[spark] val PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = + ConfigBuilder("spark.shuffle.push.maxBlockSizeToPush") .doc("The max size of an individual block to push to the remote shuffle services when push " + "based shuffle is enabled. Blocks larger than this threshold are not pushed.") + .version("3.1.0") .bytesConf(ByteUnit.KiB) .createWithDefaultString("800k") - private[spark] val PUSH_BASED_SHUFFLE_MAX_BLOCK_BATCH_SIZE = - ConfigBuilder("spark.shuffle.push.based.maxBlockBatchSize") + private[spark] val PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE = + ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") .doc("The max size of a batch of shuffle blocks to be grouped into a single push request " + "when push based shuffle is enabled.") + .version("3.1.0") .bytesConf(ByteUnit.MiB) // 2m is also the default value for TransportConf#memoryMapBytes. // Having this default to 2m will very likely make each batch of block loaded in memory with diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 2000ca88101aa..44e262d9ef0c3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -23,22 +23,23 @@ import java.nio.ByteBuffer import java.util.concurrent.ExecutorService import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} + +import com.google.common.base.Throwables + import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} -import org.apache.spark.internal.{Logging, config} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.ShuffleWriter._ -import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} import org.apache.spark.util.{ThreadUtils, Utils} -import scala.collection.mutable - /** * Obtained inside a map task to write out records to the shuffle system, and optionally * initiate the block push process to remote shuffle services if push based shuffle is enabled. @@ -64,20 +65,17 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { def getPartitionLengths(): Array[Long] - /** - * VisbleForTesting - */ + // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { new BlockPushErrorHandler() { + /** + * For a connection exception against a particular host, we will stop pushing any + * blocks to just that host and continue push blocks to other hosts. So, here push of + * all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. + */ override def shouldRetryError(t: Throwable): Boolean = { // If the block is too late, there is no need to retry it - if ((t.getMessage != null && - t.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) || - (t.getCause != null && t.getCause.getMessage != null && - t.getCause.getMessage.contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) { - return false - } - true + !Throwables.getStackTraceAsString(t).contains(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX) } } } @@ -106,8 +104,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { val dataFile = resolver.getDataFile(dep.shuffleId, mapId) val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - val maxBlockSizeToPush = conf.get(PUSH_BASED_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - val maxBlockBatchSize = conf.get(PUSH_BASED_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 + val maxBlockSizeToPush = conf.get(PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + val maxBlockBatchSize = conf.get(PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) @@ -129,7 +127,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { /** * Triggers the push. It's a separate method for testing. - * Visible for testing + * VisibleForTesting */ protected def submitTask(task: Runnable): Unit = { if (BLOCK_PUSHER_POOL != null) { @@ -188,8 +186,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // Checks if sending a new push request will exceed the max no. of blocks being pushed to a // given remote address. def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: PushRequest): Boolean = { - numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) - + request.blocks.size > maxBlocksInFlightPerAddress + (numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + + request.blocks.size) > maxBlocksInFlightPerAddress } } @@ -212,9 +210,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { val remainingBlocks = new HashSet[String]() ++= blockIds val blockPushListener = new BlockFetchingListener { - // Initiating a connection and pushing blocks to a remote shuffle service is always handled by - // the block-push-threads. We don't initiate the connection creation in the + // the block-push-threads. We should not initiate the connection creation in the // blockPushListener callbacks which are invoked by the netty eventloop because: // 1. TrasportClient.createConnection(...) blocks for connection to be established and it's // recommended to avoid any blocking operations in the eventloop; @@ -262,10 +259,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { * the shuffle data file for a PushRequest * @param blockSizes Array of block sizes * @return Array of in memory buffer for each individual block - * - * VisibleForTesting */ - protected def sliceReqBufferIntoBlockBuffers( + private def sliceReqBufferIntoBlockBuffers( reqBuffer: ManagedBuffer, blockSizes: Seq[Long]): Array[ManagedBuffer] = { if (blockSizes.size == 1) { @@ -298,12 +293,11 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { * @return true if more blocks should be pushed; false otherwise. */ private def updateStateAndCheckIfPushMore( - bytesPushed: Long, - address: BlockManagerId, - remainingBlocks: HashSet[String], - pushResult: PushResult): Boolean = synchronized { + bytesPushed: Long, + address: BlockManagerId, + remainingBlocks: HashSet[String], + pushResult: PushResult): Boolean = synchronized { remainingBlocks -= pushResult.blockId - bytesInFlight = bytesInFlight - bytesPushed numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 if (remainingBlocks.isEmpty) { @@ -376,7 +370,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { var blocks = new ArrayBuffer[(BlockId, Long)] for (reduceId <- 0 until numPartitions) { val blockSize = partitionLengths(reduceId) - logDebug(s"Block ${ShuffleBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") + logDebug( + s"Block ${ShufflePushBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") // Skip 0-length blocks and blocks that are large enough if (blockSize > 0) { val mergerId = math.min(math.floor(reduceId * 1.0 / numPartitions * numMergers), @@ -396,8 +391,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { if (blocks.nonEmpty) { // Convert the previous batch into a PushRequest requests += PushRequest(mergerLocs(currentMergerId), blocks, - new FileSegmentManagedBuffer(transportConf, dataFile, - currentReqOffset, currentReqSize)) + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) } // Start a new batch currentReqSize = 0 @@ -409,7 +403,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } // Skip blocks exceeding the size limit for push if (blockSize <= maxBlockSizeToPush) { - blocks += ((ShuffleBlockId(shuffleId, partitionId, reduceId), blockSize)) + blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSize)) // Only update currentReqOffset if the current block is the first in the request if (currentReqOffset == -1) { currentReqOffset = offset @@ -424,10 +418,19 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // Add in the final request if (blocks.nonEmpty) { requests += PushRequest(mergerLocs(currentMergerId), blocks, - new FileSegmentManagedBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) } requests } + + // Visible for testing + protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + new FileSegmentManagedBuffer(conf, dataFile, offset, length) + } } private[spark] object ShuffleWriter { @@ -452,14 +455,14 @@ private[spark] object ShuffleWriter { * @param failure exception if the push was unsuccessful; null otherwise; */ private case class PushResult( - blockId: String, - failure: Throwable + blockId: String, + failure: Throwable ) private val BLOCK_PUSHER_POOL: ExecutorService = { val conf = SparkEnv.get.conf if (Utils.isPushBasedShuffleEnabled(conf)) { - val numThreads = conf.get(PUSH_BASED_SHUFFLE_PUSHER_THREADS) + val numThreads = conf.get(PUSH_SHUFFLE_NUM_PUSH_THREADS) .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") } else { 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 7b084e73c92f9..7b8f3eb03975a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -81,6 +81,11 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } +@DeveloperApi +case class ShufflePushBlockId(shuffleId: Int, mapIndex: Integer, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapIndex + "_" + reduceId +} + @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) @@ -122,6 +127,7 @@ object BlockId { val SHUFFLE_BATCH = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r + val SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -140,6 +146,8 @@ object BlockId { ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) => + ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala index 8fbf76334efa0..a115b402fe744 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala @@ -17,23 +17,24 @@ package org.apache.spark.shuffle +import java.io.File import java.net.ConnectException import java.nio.ByteBuffer -import java.util.concurrent.ExecutorService import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark._ -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ @@ -45,10 +46,12 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ - private val conf: SparkConf = new SparkConf(loadDefaults = false) + private var conf: SparkConf = _ + private var pushedBlocks = new ArrayBuffer[String] override def beforeEach(): Unit = { super.beforeEach() + conf = new SparkConf(loadDefaults = false) MockitoAnnotations.initMocks(this) when(dependency.partitioner).thenReturn(new HashPartitioner(8)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) @@ -63,86 +66,169 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { when(blockManager.blockStoreClient).thenReturn(shuffleClient) } - test("test block push") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) - val allBlocks = new ArrayBuffer[String] + override def afterEach(): Unit = { + pushedBlocks.clear() + super.afterEach() + } + private def interceptPushedBlocksForSuccess(): Unit = { when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) .thenAnswer((invocation: InvocationOnMock) => { - val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - allBlocks ++= pushedBlocks + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - (pushedBlocks, managedBuffers).zipped.foreach((blockId, buffer) => { + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { blockFetchListener.onBlockFetchSuccess(blockId, buffer) }) }) + } + + test("Basic block push") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + interceptPushedBlocksForSuccess() + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("Large blocks are skipped for push") { + conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array(2, 2, 2, 2, 2, 2, 2, 1100)) + interceptPushedBlocksForSuccess() testWriter.initiateBlockPush( blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) + testWriter.stop(true) + } + + test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + interceptPushedBlocksForSuccess() + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(8)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + var blockPendingResponse : String = null + var listener : BlockFetchingListener = null + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + // Expecting 2 blocks + assert(blocks.length == 2) + if (blockPendingResponse == null) { + blockPendingResponse = blocks(1) + listener = blockFetchListener + // Respond with success only for the first block which will cause all the rest of the + // blocks to be deferred + blockFetchListener.onBlockFetchSuccess(blocks(0), managedBuffers(0)) + } else { + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + } + }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) - assert(allBlocks.length == dependency.partitioner.numPartitions) + assert(pushedBlocks.length == 2) + // this will trigger push of deferred blocks + listener.onBlockFetchSuccess(blockPendingResponse, mock(classOf[ManagedBuffer])) + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 8) testWriter.stop(true) } - test("error retries") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + test("Number of shuffle blocks grouped in a single push request is limited by " + + "maxBlockBatchSize") { + conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") + interceptPushedBlocksForSuccess() + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("Error retries") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) val errorHandler = testWriter.createErrorHandler() assert( !errorHandler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) assert( - errorHandler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + errorHandler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) assert (errorHandler.shouldRetryError(new Throwable())) } - test("error logging") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + test("Error logging") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) val errorHandler = testWriter.createErrorHandler() assert( !errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) - assert( - !errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) + assert(!errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) assert(errorHandler.shouldLogError(new Throwable())) } - test("connect exceptions removes all the push requests for that host") { + test("Connect exceptions removes all the push requests for that host") { when(dependency.getMergerLocs).thenReturn( Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") - val executorService = mock(classOf[ExecutorService]) - when(executorService.submit(any[Runnable]())).thenAnswer(new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - } - }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) - val allBlocks = new ArrayBuffer[String] - when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) .thenAnswer((invocation: InvocationOnMock) => { - val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - allBlocks ++= pushedBlocks + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - pushedBlocks.foreach(blockId => { + blocks.foreach(blockId => { blockFetchListener.onBlockFetchFailure( blockId, new RuntimeException(new ConnectException())) }) }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) testWriter.initiateBlockPush( blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations - assert(allBlocks.length == 4) + assert(pushedBlocks.length == 4) } private class TestShuffleWriter( - private val numPartitions: Int) extends ShuffleWriter[Int, Int] { + private val numPartitions: Int, + private val partitionLengths: Array[Long]) extends ShuffleWriter[Int, Int] { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing @@ -158,16 +244,18 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } override def getPartitionLengths(): Array[Long] = { - Array.fill(numPartitions) { - 2 - } + partitionLengths } - override protected def sliceReqBufferIntoBlockBuffers( - reqBuffer: ManagedBuffer, blockSizes: Seq[Long]) = { - Array.fill(blockSizes.length) { - new NioManagedBuffer(ByteBuffer.wrap(Array[Byte](2))) - } + override protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + val managedBuffer = mock(classOf[ManagedBuffer]) + val byteBuffer = new Array[Byte](length.toInt) + when(managedBuffer.nioByteBuffer()).thenReturn(ByteBuffer.wrap(byteBuffer)) + managedBuffer } } } From 770c25c53699b420dde1123597dc3e55cfdd5538 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 10 Nov 2020 11:17:56 -0800 Subject: [PATCH 06/35] Renamed isPushBasedShuffleEnabled to isPushShuffleEnabled and fixed the comment --- .../scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala | 2 +- .../src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 92b1756f6a01d..caef8cb011e1b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -64,7 +64,7 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { // The map task only takes care of converting the shuffle data file into multiple // block push requests and pushing the first batch of blocks to the remote shuffle // service. The remaining block pushes is handled by netty client threads. - if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { + if (Utils.isPushShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { writer.initiateBlockPush( manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], writer.getPartitionLengths(), dep, partition.index, mapId, SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 44e262d9ef0c3..118b74643feb5 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -461,7 +461,7 @@ private[spark] object ShuffleWriter { private val BLOCK_PUSHER_POOL: ExecutorService = { val conf = SparkEnv.get.conf - if (Utils.isPushBasedShuffleEnabled(conf)) { + if (Utils.isPushShuffleEnabled(conf)) { val numThreads = conf.get(PUSH_SHUFFLE_NUM_PUSH_THREADS) .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") From d429fb3a40dcfc10deb80bba9acadef87f69caa0 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 10 Nov 2020 12:44:08 -0800 Subject: [PATCH 07/35] Added Since tags all the newly introduced classes --- .../apache/spark/shuffle/ShuffleWriter.scala | 21 ++++++++++--------- .../org/apache/spark/storage/BlockId.scala | 5 +++-- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 118b74643feb5..258fd5c305b82 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -27,7 +27,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import com.google.common.base.Throwables import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} -import org.apache.spark.internal.{config, Logging} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} @@ -68,11 +69,9 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { new BlockPushErrorHandler() { - /** - * For a connection exception against a particular host, we will stop pushing any - * blocks to just that host and continue push blocks to other hosts. So, here push of - * all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. - */ + // For a connection exception against a particular host, we will stop pushing any + // blocks to just that host and continue push blocks to other hosts. So, here push of + // all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. override def shouldRetryError(t: Throwable): Boolean = { // If the block is too late, there is no need to retry it !Throwables.getStackTraceAsString(t).contains(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX) @@ -111,7 +110,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) - maxBlocksInFlightPerAddress = conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) + maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, partitionLengths, mergerLocs, transportConf, maxBlockSizeToPush, maxBlockBatchSize) @@ -382,9 +381,9 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // exceeds the max block size to push limit. This guarantees that each PushReqeust // represents continuous blocks in the shuffle file to be pushed to the same shuffle // service, and does not go beyond existing limitations. - if (currentReqSize + blockSize <= maxBlockBatchSize && - blocks.size < maxBlocksInFlightPerAddress && - mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { + if (currentReqSize + blockSize <= maxBlockBatchSize + && blocks.size < maxBlocksInFlightPerAddress + && mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { // Add current block to current batch currentReqSize += blockSize } else { @@ -442,6 +441,7 @@ private[spark] object ShuffleWriter { * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous * blocks represented in this request */ + @Since("3.1.0") case class PushRequest( address: BlockManagerId, blocks: Seq[(BlockId, Long)], @@ -454,6 +454,7 @@ private[spark] object ShuffleWriter { * @param blockId blockId * @param failure exception if the push was unsuccessful; null otherwise; */ + @Since("3.1.0") private case class PushResult( blockId: String, failure: Throwable 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 7b8f3eb03975a..2b21e63f21437 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.util.UUID import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} /** * :: DeveloperApi :: @@ -81,9 +81,10 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } +@Since("3.1.0") @DeveloperApi case class ShufflePushBlockId(shuffleId: Int, mapIndex: Integer, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapIndex + "_" + reduceId + override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId } @DeveloperApi From 3d10b2082676dec5d0f79c4a2c1df0ba000e11bb Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 11 Nov 2020 15:17:48 -0800 Subject: [PATCH 08/35] Explicitly converting ArrayBuffer to Seq so that build with SBT succeeds --- .../main/scala/org/apache/spark/shuffle/ShuffleWriter.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 258fd5c305b82..9993afb89ce5d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -389,7 +389,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } else { if (blocks.nonEmpty) { // Convert the previous batch into a PushRequest - requests += PushRequest(mergerLocs(currentMergerId), blocks, + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) } // Start a new batch @@ -416,10 +416,10 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } // Add in the final request if (blocks.nonEmpty) { - requests += PushRequest(mergerLocs(currentMergerId), blocks, + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) } - requests + requests.toSeq } // Visible for testing From c19bcf2226bdac08e6875dcaa00e6f2c53562f83 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 12 Nov 2020 08:31:24 -0800 Subject: [PATCH 09/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala Co-authored-by: wuyi --- .../src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 9993afb89ce5d..067a4e1e7999e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -412,7 +412,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } } } - offset = offset + blockSize + offset += blockSize } // Add in the final request if (blocks.nonEmpty) { From f411944b899b88f1007f85121b25e37f98d8fee6 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 12 Nov 2020 09:13:19 -0800 Subject: [PATCH 10/35] Addressed the typos and other comments --- .../apache/spark/shuffle/ShuffleWriter.scala | 31 +++++++++---------- .../org/apache/spark/storage/BlockId.scala | 2 +- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 067a4e1e7999e..9e7a8b92b3f35 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -33,7 +33,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} +import org.apache.spark.network.shuffle.{BlockFetchingListener} import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus @@ -118,9 +118,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // time won't be pushing the same ranges of shuffle partitions. pushRequests ++= Utils.randomize(requests) - val shuffleClient = SparkEnv.get.blockManager.blockStoreClient submitTask(() => { - pushUpToMax(shuffleClient) + pushUpToMax() }) } @@ -144,7 +143,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles * the data transfer between shuffle client/server. */ - private def pushUpToMax(shuffleClient: BlockStoreClient): Unit = synchronized { + private def pushUpToMax(): Unit = synchronized { // Process any outstanding deferred push requests if possible. if (deferredPushRequests.nonEmpty) { for ((remoteAddress, defReqQueue) <- deferredPushRequests) { @@ -153,7 +152,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { val request = defReqQueue.dequeue() logDebug(s"Processing deferred push request for $remoteAddress with " + s"${request.blocks.length} blocks") - sendRequest(request, shuffleClient) + sendRequest(request) if (defReqQueue.isEmpty) { deferredPushRequests -= remoteAddress } @@ -171,7 +170,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { defReqQueue.enqueue(request) deferredPushRequests(remoteAddress) = defReqQueue } else { - sendRequest(request, shuffleClient) + sendRequest(request) } } @@ -197,7 +196,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { * client thread instead of task execution thread which takes care of majority of the block * pushes. */ - private def sendRequest(request: PushRequest, shuffleClient: BlockStoreClient): Unit = { + private def sendRequest(request: PushRequest): Unit = { bytesInFlight = bytesInFlight + request.size reqsInFlight = reqsInFlight + 1 numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( @@ -222,7 +221,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { submitTask(() => { if (updateStateAndCheckIfPushMore( sizeMap(result.blockId), address, remainingBlocks, result)) { - pushUpToMax(SparkEnv.get.blockManager.blockStoreClient) + pushUpToMax() } }) } @@ -242,7 +241,8 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { handleResult(PushResult(blockId, exception)) } } - shuffleClient.pushBlocks(address.host, address.port, blockIds.toArray, + SparkEnv.get.blockManager.blockStoreClient.pushBlocks( + address.host, address.port, blockIds.toArray, sliceReqBufferIntoBlockBuffers(request.reqBuffer, request.blocks.map(_._2)), blockPushListener) } @@ -263,7 +263,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { reqBuffer: ManagedBuffer, blockSizes: Seq[Long]): Array[ManagedBuffer] = { if (blockSizes.size == 1) { - Seq(reqBuffer).toArray + Array(reqBuffer) } else { val inMemoryBuffer = reqBuffer.nioByteBuffer() val blockOffsets = new Array[Long](blockSizes.size) @@ -378,7 +378,7 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { // Start a new PushRequest if the current request goes beyond the max batch size, // or the number of blocks in the current request goes beyond the limit per destination, // or the next block push location is for a different shuffle service, or the next block - // exceeds the max block size to push limit. This guarantees that each PushReqeust + // exceeds the max block size to push limit. This guarantees that each PushRequest // represents continuous blocks in the shuffle file to be pushed to the same shuffle // service, and does not go beyond existing limitations. if (currentReqSize + blockSize <= maxBlockBatchSize @@ -394,13 +394,13 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { } // Start a new batch currentReqSize = 0 - // Set currentReqffset to -1 so we are able to distinguish between the initial value + // Set currentReqOffset to -1 so we are able to distinguish between the initial value // of currentReqOffset and when we are about to start a new batch currentReqOffset = -1 currentMergerId = mergerId blocks = new ArrayBuffer[(BlockId, Long)] } - // Skip blocks exceeding the size limit for push + // Only push blocks under the size limit if (blockSize <= maxBlockSizeToPush) { blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSize)) // Only update currentReqOffset if the current block is the first in the request @@ -455,10 +455,7 @@ private[spark] object ShuffleWriter { * @param failure exception if the push was unsuccessful; null otherwise; */ @Since("3.1.0") - private case class PushResult( - blockId: String, - failure: Throwable - ) + private case class PushResult(blockId: String, failure: Throwable) private val BLOCK_PUSHER_POOL: ExecutorService = { val conf = SparkEnv.get.conf 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 2b21e63f21437..3f69d5fc49853 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -83,7 +83,7 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten @Since("3.1.0") @DeveloperApi -case class ShufflePushBlockId(shuffleId: Int, mapIndex: Integer, reduceId: Int) extends BlockId { +case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) extends BlockId { override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId } From bd3649f4f31851a65309cb9c0590b060041068d0 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 17 Nov 2020 18:56:38 -0800 Subject: [PATCH 11/35] Moved push of data out of ShuffleWriter and addressed other minor comments --- .../org/apache/spark/executor/Executor.scala | 4 +- .../spark/internal/config/package.scala | 4 +- .../spark/shuffle/PushShuffleSupport.scala | 466 ++++++++++++++++++ .../spark/shuffle/ShuffleWriteProcessor.scala | 11 +- .../apache/spark/shuffle/ShuffleWriter.scala | 450 +---------------- ...te.scala => PushShuffleSupportSuite.scala} | 82 ++- 6 files changed, 512 insertions(+), 505 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala rename core/src/test/scala/org/apache/spark/shuffle/{ShuffleWriterSuite.scala => PushShuffleSupportSuite.scala} (76%) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6c251b54ef474..dc5b4937d89b1 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.{FetchFailedException, ShuffleWriter} +import org.apache.spark.shuffle.{FetchFailedException, PushShuffleSupport} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,7 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } - ShuffleWriter.stop + PushShuffleSupport.stop() threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly 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 2081382c9407d..63d31ac58d4b7 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 @@ -2026,9 +2026,9 @@ package object config { private[spark] val PUSH_SHUFFLE_NUM_PUSH_THREADS = ConfigBuilder("spark.shuffle.push.numPushThreads") - .doc("Specify the number of threads in the block pusher pool. These threads assist " + + .doc("Specify the number of threads in the block pusher pool. These threads assist " + "in creating connections and pushing blocks to remote shuffle services when push based " + - "shuffle is enabled. By default, the threadpool size is equal to the number of cores") + "shuffle is enabled. By default, the threadpool size is equal to the number of cores.") .version("3.1.0") .intConf .createOptional diff --git a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala new file mode 100644 index 0000000000000..7b0e302cd7843 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala @@ -0,0 +1,466 @@ +/* + * 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 java.io.File +import java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} + +import com.google.common.base.Throwables + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf +import org.apache.spark.shuffle.PushShuffleSupport._ +import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Used for pushing shuffle blocks to remote shuffle services when push shuffle is enabled. + * When push shuffle is enabled, it is created after the shuffle writer finishes writing the shuffle + * file and initiates the block push process. + * + * @param dataFile mapper generated shuffle data file + * @param partitionLengths array of shuffle block size so we can tell shuffle block + * boundaries within the shuffle file + * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle + * services to push local shuffle blocks + * @param partitionId map index of the shuffle map task + * @param conf spark configuration + */ +private[spark] class PushShuffleSupport( + dataFile: File, + partitionLengths: Array[Long], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + conf: SparkConf) extends Logging { + private[this] var maxBytesInFlight = 0L + private[this] var maxReqsInFlight = 0 + private[this] var maxBlocksInFlightPerAddress = 0 + private[this] var bytesInFlight = 0L + private[this] var reqsInFlight = 0 + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() + private[this] val pushRequests = new Queue[PushRequest] + private[this] val errorHandler = createErrorHandler() + private[this] val unreachableBlockMgrs = new HashSet[BlockManagerId]() + + initiateBlockPush() + + // VisibleForTesting + private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { + new BlockPushErrorHandler() { + // For a connection exception against a particular host, we will stop pushing any + // blocks to just that host and continue push blocks to other hosts. So, here push of + // all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. + override def shouldRetryError(t: Throwable): Boolean = { + // If the block is too late, there is no need to retry it + !Throwables.getStackTraceAsString(t).contains(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX) + } + } + } + + def initiateBlockPush(): Unit = { + val numPartitions = dep.partitioner.numPartitions + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") + + val maxBlockSizeToPush = conf.get(PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + val maxBlockBatchSize = conf.get(PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 + val mergerLocs = dep.getMergerLocs.map(loc => + BlockManagerId("", loc.host, loc.port)) + + maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) + + val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, + partitionLengths, mergerLocs, transportConf, maxBlockSizeToPush, maxBlockBatchSize) + // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same + // time won't be pushing the same ranges of shuffle partitions. + pushRequests ++= Utils.randomize(requests) + + submitTask(() => { + pushUpToMax() + }) + } + + /** + * Triggers the push. It's a separate method for testing. + * VisibleForTesting + */ + protected def submitTask(task: Runnable): Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.execute(task) + } + } + + /** + * Since multiple netty client threads could potentially be calling pushUpToMax for the same + * mapper, we synchronize access to this method so that only one thread can push blocks for + * a given mapper. This helps to simplify access to the shared states. The down side of this + * is that we could unnecessarily block other mappers' block pushes if all netty client threads + * are occupied by block pushes from the same mapper. + * + * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles + * the data transfer between shuffle client/server. + */ + private def pushUpToMax(): Unit = synchronized { + // Process any outstanding deferred push requests if possible. + if (deferredPushRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredPushRequests) { + while (isRemoteBlockPushable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred push request for $remoteAddress with " + + s"${request.blocks.length} blocks") + sendRequest(request) + if (defReqQueue.isEmpty) { + deferredPushRequests -= remoteAddress + } + } + } + } + + // Process any regular push requests if possible. + while (isRemoteBlockPushable(pushRequests)) { + val request = pushRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") + val defReqQueue = deferredPushRequests.getOrElse(remoteAddress, new Queue[PushRequest]()) + defReqQueue.enqueue(request) + deferredPushRequests(remoteAddress) = defReqQueue + } else { + sendRequest(request) + } + } + + def isRemoteBlockPushable(pushReqQueue: Queue[PushRequest]): Boolean = { + pushReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + pushReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new push request will exceed the max no. of blocks being pushed to a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: PushRequest): Boolean = { + (numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + + request.blocks.size) > maxBlocksInFlightPerAddress + } + } + + /** + * Push blocks to remote shuffle server. The callback listener will invoke #pushUpToMax again + * to trigger pushing the next batch of blocks once some block transfer is done in the current + * batch. This way, we decouple the map task from the block push process, since it is netty + * client thread instead of task execution thread which takes care of majority of the block + * pushes. + */ + private def sendRequest(request: PushRequest): Unit = { + bytesInFlight = bytesInFlight + request.size + reqsInFlight = reqsInFlight + 1 + numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( + request.address, 0) + request.blocks.length + + val sizeMap = request.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val address = request.address + val blockIds = request.blocks.map(_._1.toString) + val remainingBlocks = new HashSet[String]() ++= blockIds + + val blockPushListener = new BlockFetchingListener { + // Initiating a connection and pushing blocks to a remote shuffle service is always handled by + // the block-push-threads. We should not initiate the connection creation in the + // blockPushListener callbacks which are invoked by the netty eventloop because: + // 1. TrasportClient.createConnection(...) blocks for connection to be established and it's + // recommended to avoid any blocking operations in the eventloop; + // 2. The actual connection creation is a task that gets added to the task queue of another + // eventloop which could have eventloops eventually blocking each other. + // Once the blockPushListener is notified of the block push success or failure, we + // just delegate it to block-push-threads. + def handleResult(result: PushResult): Unit = { + submitTask(() => { + if (updateStateAndCheckIfPushMore( + sizeMap(result.blockId), address, remainingBlocks, result)) { + pushUpToMax() + } + }) + } + + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + logTrace(s"Push for block $blockId to $address successful.") + handleResult(PushResult(blockId, null)) + } + + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + // check the message or it's cause to see it needs to be logged. + if (!errorHandler.shouldLogError(exception)) { + logTrace(s"Pushing block $blockId to $address failed.", exception) + } else { + logWarning(s"Pushing block $blockId to $address failed.", exception) + } + handleResult(PushResult(blockId, exception)) + } + } + SparkEnv.get.blockManager.blockStoreClient.pushBlocks( + address.host, address.port, blockIds.toArray, + sliceReqBufferIntoBlockBuffers(request.reqBuffer, request.blocks.map(_._2)), + blockPushListener) + } + + /** + * Given the ManagedBuffer representing all the continuous blocks inside the shuffle data file + * for a PushRequest and an array of individual block sizes, load the buffer from disk into + * memory and slice it into multiple smaller buffers representing each block. + * + * With nio ByteBuffer, the individual block buffers share data with the initial in memory + * buffer loaded from disk. Thus only one copy of the block data is kept in memory. + * @param reqBuffer A {{FileSegmentManagedBuffer}} representing all the continuous blocks in + * the shuffle data file for a PushRequest + * @param blockSizes Array of block sizes + * @return Array of in memory buffer for each individual block + */ + private def sliceReqBufferIntoBlockBuffers( + reqBuffer: ManagedBuffer, + blockSizes: Seq[Long]): Array[ManagedBuffer] = { + if (blockSizes.size == 1) { + Array(reqBuffer) + } else { + val inMemoryBuffer = reqBuffer.nioByteBuffer() + val blockOffsets = new Array[Long](blockSizes.size) + var offset = 0L + for (index <- blockSizes.indices) { + blockOffsets(index) = offset + offset += blockSizes(index) + } + blockOffsets.zip(blockSizes).map { + case (offset, size) => + new NioManagedBuffer(inMemoryBuffer.duplicate() + .position(offset.toInt) + .limit((offset + size).toInt).asInstanceOf[ByteBuffer].slice()) + }.toArray + } + } + + /** + * Updates the stats and based on the previous push result decides whether to push more blocks + * or stop. + * + * @param bytesPushed number of bytes pushed. + * @param address address of the remote service + * @param remainingBlocks remaining blocks + * @param pushResult result of the last push + * @return true if more blocks should be pushed; false otherwise. + */ + private def updateStateAndCheckIfPushMore( + bytesPushed: Long, + address: BlockManagerId, + remainingBlocks: HashSet[String], + pushResult: PushResult): Boolean = synchronized { + remainingBlocks -= pushResult.blockId + bytesInFlight = bytesInFlight - bytesPushed + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + if (remainingBlocks.isEmpty) { + reqsInFlight = reqsInFlight - 1 + } + if (pushResult.failure != null && pushResult.failure.getCause != null && + pushResult.failure.getCause.isInstanceOf[ConnectException]) { + // Remove all the blocks for this address just once because removing from pushRequests + // is expensive. If there is a ConnectException for the first block, all the subsequent + // blocks to that address will fail, so should avoid removing multiple times. + if (!unreachableBlockMgrs.contains(address)) { + var removed = 0 + unreachableBlockMgrs.add(address) + removed += pushRequests.dequeueAll(req => req.address == address).length + val droppedReq = deferredPushRequests.remove(address) + if (droppedReq.isDefined) { + removed += droppedReq.get.length + } + logWarning(s"Received a ConnectException from $address. " + + s"Dropping push of $removed blocks and " + + s"not pushing any more blocks to this address.") + } + } + if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { + logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") + return false + } else { + remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) + } + } + + /** + * Convert the shuffle data file of the current mapper into a list of PushRequest. Basically, + * continuous blocks in the shuffle file are grouped into a single request to allow more + * efficient read of the block data. Each mapper for a given shuffle will receive the same + * list of BlockManagerIds as the target location to push the blocks to. All mappers in the + * same shuffle will map shuffle partition ranges to individual target locations in a consistent + * manner to make sure each target location receives shuffle blocks belonging to the same set + * of partition ranges. 0-length blocks and blocks that are large enough will be skipped. + * + * @param numPartitions Number of shuffle partitions in the shuffle file + * @param partitionId map index of the current mapper + * @param shuffleId ShuffleId of current shuffle + * @param dataFile Shuffle data file + * @param partitionLengths array of sizes of blocks in the shuffle data file + * @param mergerLocs Target locations to push blocks to + * @param transportConf TransportConf used to create FileSegmentManagedBuffer + * @param maxBlockSizeToPush Max size of individual blocks that will be pushed. Blocks larger + * than this threshold will be skipped. + * @param maxBlockBatchSize Max size of a batch of shuffle blocks to be grouped into a single + * request + * @return List of the PushRequest, randomly shuffled. + */ + private def prepareBlockPushRequests( + numPartitions: Int, + partitionId: Int, + shuffleId: Int, + dataFile: File, + partitionLengths: Array[Long], + mergerLocs: Seq[BlockManagerId], + transportConf: TransportConf, + maxBlockSizeToPush: Long, + maxBlockBatchSize: Long): Seq[PushRequest] = { + var offset = 0L + var currentReqSize = 0L + var currentReqOffset = 0L + var currentMergerId = 0 + val numMergers = mergerLocs.length + val requests = new ArrayBuffer[PushRequest] + var blocks = new ArrayBuffer[(BlockId, Long)] + for (reduceId <- 0 until numPartitions) { + val blockSize = partitionLengths(reduceId) + logDebug( + s"Block ${ShufflePushBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") + // Skip 0-length blocks and blocks that are large enough + if (blockSize > 0) { + val mergerId = math.min(math.floor(reduceId * 1.0 / numPartitions * numMergers), + numMergers - 1).asInstanceOf[Int] + // Start a new PushRequest if the current request goes beyond the max batch size, + // or the number of blocks in the current request goes beyond the limit per destination, + // or the next block push location is for a different shuffle service, or the next block + // exceeds the max block size to push limit. This guarantees that each PushRequest + // represents continuous blocks in the shuffle file to be pushed to the same shuffle + // service, and does not go beyond existing limitations. + if (currentReqSize + blockSize <= maxBlockBatchSize + && blocks.size < maxBlocksInFlightPerAddress + && mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { + // Add current block to current batch + currentReqSize += blockSize + } else { + if (blocks.nonEmpty) { + // Convert the previous batch into a PushRequest + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + } + // Start a new batch + currentReqSize = 0 + // Set currentReqOffset to -1 so we are able to distinguish between the initial value + // of currentReqOffset and when we are about to start a new batch + currentReqOffset = -1 + currentMergerId = mergerId + blocks = new ArrayBuffer[(BlockId, Long)] + } + // Only push blocks under the size limit + if (blockSize <= maxBlockSizeToPush) { + blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSize)) + // Only update currentReqOffset if the current block is the first in the request + if (currentReqOffset == -1) { + currentReqOffset = offset + } + if (currentReqSize == 0) { + currentReqSize += blockSize + } + } + } + offset += blockSize + } + // Add in the final request + if (blocks.nonEmpty) { + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + } + requests.toSeq + } + + // Visible for testing + protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + new FileSegmentManagedBuffer(conf, dataFile, offset, length) + } +} + +private[spark] object PushShuffleSupport { + + /** + * A request to push blocks to a remote shuffle service + * @param address remote shuffle service location to push blocks to + * @param blocks list of block IDs and their sizes + * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous + * blocks represented in this request + */ + @Since("3.1.0") + case class PushRequest( + address: BlockManagerId, + blocks: Seq[(BlockId, Long)], + reqBuffer: ManagedBuffer) { + val size = blocks.map(_._2).sum + } + + /** + * Result of the block push. + * @param blockId blockId + * @param failure exception if the push was unsuccessful; null otherwise; + */ + @Since("3.1.0") + private case class PushResult(blockId: String, failure: Throwable) + + private val BLOCK_PUSHER_POOL: ExecutorService = { + val conf = SparkEnv.get.conf + if (Utils.isPushShuffleEnabled(conf)) { + val numThreads = conf.get(PUSH_SHUFFLE_NUM_PUSH_THREADS) + .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) + ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") + } else { + null + } + } + + /** + * Stop the shuffle writer pool if it isn't null. + */ + private[spark] def stop() : Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.shutdown() + } + } +} + diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index caef8cb011e1b..b88db3b320713 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -62,12 +62,13 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { if (mapStatus.isDefined) { // Initiate shuffle push process if push based shuffle is enabled // The map task only takes care of converting the shuffle data file into multiple - // block push requests and pushing the first batch of blocks to the remote shuffle - // service. The remaining block pushes is handled by netty client threads. + // block push requests. It delegates pushing the blocks to a different thread-pool - + // PushShuffleSupport.BLOCK_PUSHER_POOL. if (Utils.isPushShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { - writer.initiateBlockPush( - manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], - writer.getPartitionLengths(), dep, partition.index, mapId, SparkEnv.get.conf) + val dataFile = manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver] + .getDataFile(dep.shuffleId, mapId) + new PushShuffleSupport(dataFile, writer.getPartitionLengths(), dep, + partition.index, SparkEnv.get.conf) } } mapStatus.get diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 9e7a8b92b3f35..a279b4c8f42f4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -17,46 +17,14 @@ package org.apache.spark.shuffle -import java.io.{File, IOException} -import java.net.ConnectException -import java.nio.ByteBuffer -import java.util.concurrent.ExecutorService +import java.io.IOException -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} - -import com.google.common.base.Throwables - -import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{BlockFetchingListener} -import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler -import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus -import org.apache.spark.shuffle.ShuffleWriter._ -import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} -import org.apache.spark.util.{ThreadUtils, Utils} /** - * Obtained inside a map task to write out records to the shuffle system, and optionally - * initiate the block push process to remote shuffle services if push based shuffle is enabled. + * Obtained inside a map task to write out records to the shuffle system. */ -private[spark] abstract class ShuffleWriter[K, V] extends Logging { - private[this] var maxBytesInFlight = 0L - private[this] var maxReqsInFlight = 0 - private[this] var maxBlocksInFlightPerAddress = 0 - private[this] var bytesInFlight = 0L - private[this] var reqsInFlight = 0 - private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() - private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() - private[this] val pushRequests = new Queue[PushRequest] - private[this] val errorHandler = createErrorHandler() - private[this] val unreachableBlockMgrs = new HashSet[BlockManagerId]() - +private[spark] abstract class ShuffleWriter[K, V] { /** Write a sequence of records to this task's output */ @throws[IOException] def write(records: Iterator[Product2[K, V]]): Unit @@ -64,416 +32,6 @@ private[spark] abstract class ShuffleWriter[K, V] extends Logging { /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] + /** Get the lengths of each partition */ def getPartitionLengths(): Array[Long] - - // VisibleForTesting - private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { - new BlockPushErrorHandler() { - // For a connection exception against a particular host, we will stop pushing any - // blocks to just that host and continue push blocks to other hosts. So, here push of - // all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. - override def shouldRetryError(t: Throwable): Boolean = { - // If the block is too late, there is no need to retry it - !Throwables.getStackTraceAsString(t).contains(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX) - } - } - } - - /** - * Initiate the block push process. This will be invoked after the shuffle writer - * finishes writing the shuffle file if push based shuffle is enabled. - * - * @param resolver block resolver used to locate mapper generated shuffle file - * @param partitionLengths array of shuffle block size so we can tell shuffle block - * boundaries within the shuffle file - * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle - * services to push local shuffle blocks - * @param partitionId map index of the shuffle map task - * @param mapId mapId of the shuffle map task - * @param conf spark configuration - */ - def initiateBlockPush( - resolver: IndexShuffleBlockResolver, - partitionLengths: Array[Long], - dep: ShuffleDependency[_, _, _], - partitionId: Int, - mapId: Long, - conf: SparkConf): Unit = { - val numPartitions = dep.partitioner.numPartitions - val dataFile = resolver.getDataFile(dep.shuffleId, mapId) - val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - - val maxBlockSizeToPush = conf.get(PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - val maxBlockBatchSize = conf.get(PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 - val mergerLocs = dep.getMergerLocs.map(loc => - BlockManagerId("", loc.host, loc.port)) - - maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 - maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) - maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) - - val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, - partitionLengths, mergerLocs, transportConf, maxBlockSizeToPush, maxBlockBatchSize) - // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same - // time won't be pushing the same ranges of shuffle partitions. - pushRequests ++= Utils.randomize(requests) - - submitTask(() => { - pushUpToMax() - }) - } - - /** - * Triggers the push. It's a separate method for testing. - * VisibleForTesting - */ - protected def submitTask(task: Runnable): Unit = { - if (BLOCK_PUSHER_POOL != null) { - BLOCK_PUSHER_POOL.execute(task) - } - } - - /** - * Since multiple netty client threads could potentially be calling pushUpToMax for the same - * mapper, we synchronize access to this method so that only one thread can push blocks for - * a given mapper. This helps to simplify access to the shared states. The down side of this - * is that we could unnecessarily block other mappers' block pushes if all netty client threads - * are occupied by block pushes from the same mapper. - * - * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles - * the data transfer between shuffle client/server. - */ - private def pushUpToMax(): Unit = synchronized { - // Process any outstanding deferred push requests if possible. - if (deferredPushRequests.nonEmpty) { - for ((remoteAddress, defReqQueue) <- deferredPushRequests) { - while (isRemoteBlockPushable(defReqQueue) && - !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { - val request = defReqQueue.dequeue() - logDebug(s"Processing deferred push request for $remoteAddress with " - + s"${request.blocks.length} blocks") - sendRequest(request) - if (defReqQueue.isEmpty) { - deferredPushRequests -= remoteAddress - } - } - } - } - - // Process any regular push requests if possible. - while (isRemoteBlockPushable(pushRequests)) { - val request = pushRequests.dequeue() - val remoteAddress = request.address - if (isRemoteAddressMaxedOut(remoteAddress, request)) { - logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") - val defReqQueue = deferredPushRequests.getOrElse(remoteAddress, new Queue[PushRequest]()) - defReqQueue.enqueue(request) - deferredPushRequests(remoteAddress) = defReqQueue - } else { - sendRequest(request) - } - } - - def isRemoteBlockPushable(pushReqQueue: Queue[PushRequest]): Boolean = { - pushReqQueue.nonEmpty && - (bytesInFlight == 0 || - (reqsInFlight + 1 <= maxReqsInFlight && - bytesInFlight + pushReqQueue.front.size <= maxBytesInFlight)) - } - - // Checks if sending a new push request will exceed the max no. of blocks being pushed to a - // given remote address. - def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: PushRequest): Boolean = { - (numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) - + request.blocks.size) > maxBlocksInFlightPerAddress - } - } - - /** - * Push blocks to remote shuffle server. The callback listener will invoke #pushUpToMax again - * to trigger pushing the next batch of blocks once some block transfer is done in the current - * batch. This way, we decouple the map task from the block push process, since it is netty - * client thread instead of task execution thread which takes care of majority of the block - * pushes. - */ - private def sendRequest(request: PushRequest): Unit = { - bytesInFlight = bytesInFlight + request.size - reqsInFlight = reqsInFlight + 1 - numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( - request.address, 0) + request.blocks.length - - val sizeMap = request.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap - val address = request.address - val blockIds = request.blocks.map(_._1.toString) - val remainingBlocks = new HashSet[String]() ++= blockIds - - val blockPushListener = new BlockFetchingListener { - // Initiating a connection and pushing blocks to a remote shuffle service is always handled by - // the block-push-threads. We should not initiate the connection creation in the - // blockPushListener callbacks which are invoked by the netty eventloop because: - // 1. TrasportClient.createConnection(...) blocks for connection to be established and it's - // recommended to avoid any blocking operations in the eventloop; - // 2. The actual connection creation is a task that gets added to the task queue of another - // eventloop which could have eventloops eventually blocking each other. - // Once the blockPushListener is notified of the block push success or failure, we - // just delegate it to block-push-threads. - def handleResult(result: PushResult): Unit = { - submitTask(() => { - if (updateStateAndCheckIfPushMore( - sizeMap(result.blockId), address, remainingBlocks, result)) { - pushUpToMax() - } - }) - } - - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - logTrace(s"Push for block $blockId to $address successful.") - handleResult(PushResult(blockId, null)) - } - - override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - // check the message or it's cause to see it needs to be logged. - if (!errorHandler.shouldLogError(exception)) { - logTrace(s"Pushing block $blockId to $address failed.", exception) - } else { - logWarning(s"Pushing block $blockId to $address failed.", exception) - } - handleResult(PushResult(blockId, exception)) - } - } - SparkEnv.get.blockManager.blockStoreClient.pushBlocks( - address.host, address.port, blockIds.toArray, - sliceReqBufferIntoBlockBuffers(request.reqBuffer, request.blocks.map(_._2)), - blockPushListener) - } - - /** - * Given the ManagedBuffer representing all the continuous blocks inside the shuffle data file - * for a PushRequest and an array of individual block sizes, load the buffer from disk into - * memory and slice it into multiple smaller buffers representing each block. - * - * With nio ByteBuffer, the individual block buffers share data with the initial in memory - * buffer loaded from disk. Thus only one copy of the block data is kept in memory. - * @param reqBuffer A {{FileSegmentManagedBuffer}} representing all the continuous blocks in - * the shuffle data file for a PushRequest - * @param blockSizes Array of block sizes - * @return Array of in memory buffer for each individual block - */ - private def sliceReqBufferIntoBlockBuffers( - reqBuffer: ManagedBuffer, - blockSizes: Seq[Long]): Array[ManagedBuffer] = { - if (blockSizes.size == 1) { - Array(reqBuffer) - } else { - val inMemoryBuffer = reqBuffer.nioByteBuffer() - val blockOffsets = new Array[Long](blockSizes.size) - var offset = 0L - for (index <- blockSizes.indices) { - blockOffsets(index) = offset - offset += blockSizes(index) - } - blockOffsets.zip(blockSizes).map { - case (offset, size) => - new NioManagedBuffer(inMemoryBuffer.duplicate() - .position(offset.toInt) - .limit((offset + size).toInt).asInstanceOf[ByteBuffer].slice()) - }.toArray - } - } - - /** - * Updates the stats and based on the previous push result decides whether to push more blocks - * or stop. - * - * @param bytesPushed number of bytes pushed. - * @param address address of the remote service - * @param remainingBlocks remaining blocks - * @param pushResult result of the last push - * @return true if more blocks should be pushed; false otherwise. - */ - private def updateStateAndCheckIfPushMore( - bytesPushed: Long, - address: BlockManagerId, - remainingBlocks: HashSet[String], - pushResult: PushResult): Boolean = synchronized { - remainingBlocks -= pushResult.blockId - bytesInFlight = bytesInFlight - bytesPushed - numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 - if (remainingBlocks.isEmpty) { - reqsInFlight = reqsInFlight - 1 - } - if (pushResult.failure != null && pushResult.failure.getCause != null && - pushResult.failure.getCause.isInstanceOf[ConnectException]) { - // Remove all the blocks for this address just once because removing from pushRequests - // is expensive. If there is a ConnectException for the first block, all the subsequent - // blocks to that address will fail, so should avoid removing multiple times. - if (!unreachableBlockMgrs.contains(address)) { - var removed = 0 - unreachableBlockMgrs.add(address) - removed += pushRequests.dequeueAll(req => req.address == address).length - val droppedReq = deferredPushRequests.remove(address) - if (droppedReq.isDefined) { - removed += droppedReq.get.length - } - logWarning(s"Received a ConnectException from $address. " + - s"Dropping push of $removed blocks and " + - s"not pushing any more blocks to this address.") - } - } - if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { - logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") - return false - } else { - remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) - } - } - - /** - * Convert the shuffle data file of the current mapper into a list of PushRequest. Basically, - * continuous blocks in the shuffle file are grouped into a single request to allow more - * efficient read of the block data. Each mapper for a given shuffle will receive the same - * list of BlockManagerIds as the target location to push the blocks to. All mappers in the - * same shuffle will map shuffle partition ranges to individual target locations in a consistent - * manner to make sure each target location receives shuffle blocks belonging to the same set - * of partition ranges. 0-length blocks and blocks that are large enough will be skipped. - * - * @param numPartitions Number of shuffle partitions in the shuffle file - * @param partitionId map index of the current mapper - * @param shuffleId ShuffleId of current shuffle - * @param dataFile Shuffle data file - * @param partitionLengths array of sizes of blocks in the shuffle data file - * @param mergerLocs Target locations to push blocks to - * @param transportConf TransportConf used to create FileSegmentManagedBuffer - * @param maxBlockSizeToPush Max size of individual blocks that will be pushed. Blocks larger - * than this threshold will be skipped. - * @param maxBlockBatchSize Max size of a batch of shuffle blocks to be grouped into a single - * request - * @return List of the PushRequest, randomly shuffled. - */ - private def prepareBlockPushRequests( - numPartitions: Int, - partitionId: Int, - shuffleId: Int, - dataFile: File, - partitionLengths: Array[Long], - mergerLocs: Seq[BlockManagerId], - transportConf: TransportConf, - maxBlockSizeToPush: Long, - maxBlockBatchSize: Long): Seq[PushRequest] = { - var offset = 0L - var currentReqSize = 0L - var currentReqOffset = 0L - var currentMergerId = 0 - val numMergers = mergerLocs.length - val requests = new ArrayBuffer[PushRequest] - var blocks = new ArrayBuffer[(BlockId, Long)] - for (reduceId <- 0 until numPartitions) { - val blockSize = partitionLengths(reduceId) - logDebug( - s"Block ${ShufflePushBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") - // Skip 0-length blocks and blocks that are large enough - if (blockSize > 0) { - val mergerId = math.min(math.floor(reduceId * 1.0 / numPartitions * numMergers), - numMergers - 1).asInstanceOf[Int] - // Start a new PushRequest if the current request goes beyond the max batch size, - // or the number of blocks in the current request goes beyond the limit per destination, - // or the next block push location is for a different shuffle service, or the next block - // exceeds the max block size to push limit. This guarantees that each PushRequest - // represents continuous blocks in the shuffle file to be pushed to the same shuffle - // service, and does not go beyond existing limitations. - if (currentReqSize + blockSize <= maxBlockBatchSize - && blocks.size < maxBlocksInFlightPerAddress - && mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { - // Add current block to current batch - currentReqSize += blockSize - } else { - if (blocks.nonEmpty) { - // Convert the previous batch into a PushRequest - requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, - createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) - } - // Start a new batch - currentReqSize = 0 - // Set currentReqOffset to -1 so we are able to distinguish between the initial value - // of currentReqOffset and when we are about to start a new batch - currentReqOffset = -1 - currentMergerId = mergerId - blocks = new ArrayBuffer[(BlockId, Long)] - } - // Only push blocks under the size limit - if (blockSize <= maxBlockSizeToPush) { - blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSize)) - // Only update currentReqOffset if the current block is the first in the request - if (currentReqOffset == -1) { - currentReqOffset = offset - } - if (currentReqSize == 0) { - currentReqSize += blockSize - } - } - } - offset += blockSize - } - // Add in the final request - if (blocks.nonEmpty) { - requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, - createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) - } - requests.toSeq - } - - // Visible for testing - protected def createRequestBuffer( - conf: TransportConf, - dataFile: File, - offset: Long, - length: Long): ManagedBuffer = { - new FileSegmentManagedBuffer(conf, dataFile, offset, length) - } -} - -private[spark] object ShuffleWriter { - - /** - * A request to push blocks to a remote shuffle service - * @param address remote shuffle service location to push blocks to - * @param blocks list of block IDs and their sizes - * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous - * blocks represented in this request - */ - @Since("3.1.0") - case class PushRequest( - address: BlockManagerId, - blocks: Seq[(BlockId, Long)], - reqBuffer: ManagedBuffer) { - val size = blocks.map(_._2).sum - } - - /** - * Result of the block push. - * @param blockId blockId - * @param failure exception if the push was unsuccessful; null otherwise; - */ - @Since("3.1.0") - private case class PushResult(blockId: String, failure: Throwable) - - private val BLOCK_PUSHER_POOL: ExecutorService = { - val conf = SparkEnv.get.conf - if (Utils.isPushShuffleEnabled(conf)) { - val numThreads = conf.get(PUSH_SHUFFLE_NUM_PUSH_THREADS) - .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) - ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") - } else { - null - } - } - - /** - * Stop the shuffle writer pool if it isn't null. - */ - private[spark] def stop : Unit = { - if (BLOCK_PUSHER_POOL != null) { - BLOCK_PUSHER_POOL.shutdown() - } - } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala similarity index 76% rename from core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala index a115b402fe744..e859810f79f4e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala @@ -35,14 +35,12 @@ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf -import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ -class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { +class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ @@ -85,41 +83,35 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Basic block push") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) interceptPushedBlocksForSuccess() - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) + PushShuffleSupport.stop() } test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array(2, 2, 2, 2, 2, 2, 2, 1100)) interceptPushedBlocksForSuccess() - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), + dependency, 0, conf) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) - testWriter.stop(true) + PushShuffleSupport.stop() } test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) + PushShuffleSupport.stop() } test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { @@ -146,10 +138,8 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { }) } }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 2) @@ -158,27 +148,25 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 8) - testWriter.stop(true) + PushShuffleSupport.stop() } test("Number of shuffle blocks grouped in a single push request is limited by " + "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0, conf) verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) + PushShuffleSupport.stop() } test("Error retries") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - val errorHandler = testWriter.createErrorHandler() + val pushShuffleSupport = new PushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + val errorHandler = pushShuffleSupport.createErrorHandler() assert( !errorHandler.shouldRetryError(new RuntimeException( new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) @@ -190,9 +178,9 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Error logging") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - val errorHandler = testWriter.createErrorHandler() + val pushShuffleSupport = new PushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + val errorHandler = pushShuffleSupport.createErrorHandler() assert( !errorHandler.shouldLogError(new RuntimeException( new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) @@ -216,34 +204,28 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + new TestPushShuffleSupport(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations assert(pushedBlocks.length == 4) } - private class TestShuffleWriter( - private val numPartitions: Int, - private val partitionLengths: Array[Long]) extends ShuffleWriter[Int, Int] { + private class TestPushShuffleSupport( + dataFile: File, + partitionLengths: Array[Long], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + conf: SparkConf) + extends PushShuffleSupport(dataFile, partitionLengths, dep, partitionId, conf) { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing task.run() } - /** Write a sequence of records to this task's output */ - override def write(records: Iterator[Product2[Int, Int]]): Unit = {} - - /** Close this writer, passing along whether the map completed */ - override def stop(success: Boolean): Option[MapStatus] = { - Option.empty - } - - override def getPartitionLengths(): Array[Long] = { + def getPartitionLengths(): Array[Long] = { partitionLengths } From bf4b277637a1043cc9dc569973f66a0dc801b0df Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 8 May 2020 13:25:31 -0700 Subject: [PATCH 12/35] LIHADOOP-48527 Enable Shuffle writer to push blocks to remote shuffle services. It also includes the below commits: 1. commit 1d63f190fdd7c6839161a048abb29084be97efba Author: Chandni Singh Date: Thu Mar 12 19:32:31 2020 -0700 LIHADOOP-51112 Create connections outside netty event loop RB=1977902 BUG=LIHADOOP-51112 G=superfriends-reviewers R=mshen 2. commit 16800055ad9188850014726f4b3bf69abb354aec Author: Chandni Singh Date: Mon Mar 16 17:31:20 2020 -0700 LIHADOOP-51891 prepareBlockPushRequests in ShuffleWriter uses parameters that are initialized later RB=2010539 BUG=LIHADOOP-51891 G=superfriends-reviewers R=mshen --- .../BypassMergeSortShuffleWriterSuite.scala | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index f8474022867f4..01a1ddee719fd 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.shuffle.sort import java.io.File import java.util.UUID +import java.util.concurrent.{Semaphore, TimeUnit} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -27,11 +28,14 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents @@ -71,6 +75,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) + when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) @@ -236,4 +242,49 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } + + // TODO: Move this to ShuffleWriterSuite. It passes in the IDE but fails consistently with mvn. + ignore("test block push") { + def records: Iterator[(Int, Int)] = Iterator((1, 1), (2, 2), (3, 3)) + + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + shuffleHandle, + 0, // MapId + conf, + taskContext.taskMetrics().shuffleWriteMetrics, + shuffleExecutorComponents + ) + conf.set("spark.shuffle.push.based.enabled", "true") + conf.set("spark.shuffle.service.enabled", "true") + val mockEnv = mock(classOf[SparkEnv]) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.blockManager).thenReturn(blockManager) + val shuffleClient = mock(classOf[BlockStoreClient]) + when(blockManager.blockStoreClient).thenReturn(shuffleClient) + SparkEnv.set(mockEnv) + + val sem = new Semaphore(0) + var blockIds : Array[String] = null + when(shuffleClient.pushBlocks(any(), anyInt(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + blockIds = invocation.getArguments()(2).asInstanceOf[Array[String]] + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = + invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + (blockIds, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + sem.release() + }) + + writer.write(records) + writer.initiateBlockPush(blockResolver, writer.getPartitionLengths(), dependency, 0, 0, conf) + + sem.tryAcquire(1, 500, TimeUnit.MILLISECONDS) + for (i <- 1 to 3) { + assert(blockIds(i - 1) === ShuffleBlockId(0, 0, i).toString) + } + writer.stop(true) + } } From a8a350c422b2f8423c671879019b59e2791d2cde Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 23 Jun 2020 23:10:07 -0700 Subject: [PATCH 13/35] LIHADOOP-54370 Not to retry on certain exceptions when pushing blocks RB=2166258 BUG=LIHADOOP-54370 G=spark-reviewers R=mshen,yezhou A=mshen --- .../spark/shuffle/ShuffleWriterSuite.scala | 173 ++++++++++++++++++ .../BypassMergeSortShuffleWriterSuite.scala | 51 ------ 2 files changed, 173 insertions(+), 51 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala new file mode 100644 index 0000000000000..8fbf76334efa0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala @@ -0,0 +1,173 @@ +/* + * 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 java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService + +import scala.collection.mutable.ArrayBuffer + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage._ + +class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ + @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ + + private val conf: SparkConf = new SparkConf(loadDefaults = false) + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + when(dependency.partitioner).thenReturn(new HashPartitioner(8)) + when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) + conf.set("spark.shuffle.push.based.enabled", "true") + conf.set("spark.shuffle.service.enabled", "true") + // Set the env because the shuffler writer gets the shuffle client instance from the env. + val mockEnv = mock(classOf[SparkEnv]) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.blockManager).thenReturn(blockManager) + SparkEnv.set(mockEnv) + when(blockManager.blockStoreClient).thenReturn(shuffleClient) + } + + test("test block push") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val allBlocks = new ArrayBuffer[String] + + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + allBlocks ++= pushedBlocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + (pushedBlocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(allBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("error retries") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val errorHandler = testWriter.createErrorHandler() + assert( + !errorHandler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) + assert( + errorHandler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + assert (errorHandler.shouldRetryError(new Throwable())) + } + + test("error logging") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val errorHandler = testWriter.createErrorHandler() + assert( + !errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + assert( + !errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + assert(errorHandler.shouldLogError(new Throwable())) + } + + test("connect exceptions removes all the push requests for that host") { + when(dependency.getMergerLocs).thenReturn( + Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + val executorService = mock(classOf[ExecutorService]) + when(executorService.submit(any[Runnable]())).thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + } + }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + val allBlocks = new ArrayBuffer[String] + + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + allBlocks ++= pushedBlocks + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + pushedBlocks.foreach(blockId => { + blockFetchListener.onBlockFetchFailure( + blockId, new RuntimeException(new ConnectException())) + }) + }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(2)) + .pushBlocks(any(), any(), any(), any(), any()) + // 2 blocks for each merger locations + assert(allBlocks.length == 4) + } + + private class TestShuffleWriter( + private val numPartitions: Int) extends ShuffleWriter[Int, Int] { + + override protected def submitTask(task: Runnable): Unit = { + // Making this synchronous for testing + task.run() + } + + /** Write a sequence of records to this task's output */ + override def write(records: Iterator[Product2[Int, Int]]): Unit = {} + + /** Close this writer, passing along whether the map completed */ + override def stop(success: Boolean): Option[MapStatus] = { + Option.empty + } + + override def getPartitionLengths(): Array[Long] = { + Array.fill(numPartitions) { + 2 + } + } + + override protected def sliceReqBufferIntoBlockBuffers( + reqBuffer: ManagedBuffer, blockSizes: Seq[Long]) = { + Array.fill(blockSizes.length) { + new NioManagedBuffer(ByteBuffer.wrap(Array[Byte](2))) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 01a1ddee719fd..f8474022867f4 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.shuffle.sort import java.io.File import java.util.UUID -import java.util.concurrent.{Semaphore, TimeUnit} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -28,14 +27,11 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents @@ -75,8 +71,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) - when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) - when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) @@ -242,49 +236,4 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } - - // TODO: Move this to ShuffleWriterSuite. It passes in the IDE but fails consistently with mvn. - ignore("test block push") { - def records: Iterator[(Int, Int)] = Iterator((1, 1), (2, 2), (3, 3)) - - val writer = new BypassMergeSortShuffleWriter[Int, Int]( - blockManager, - shuffleHandle, - 0, // MapId - conf, - taskContext.taskMetrics().shuffleWriteMetrics, - shuffleExecutorComponents - ) - conf.set("spark.shuffle.push.based.enabled", "true") - conf.set("spark.shuffle.service.enabled", "true") - val mockEnv = mock(classOf[SparkEnv]) - when(mockEnv.conf).thenReturn(conf) - when(mockEnv.blockManager).thenReturn(blockManager) - val shuffleClient = mock(classOf[BlockStoreClient]) - when(blockManager.blockStoreClient).thenReturn(shuffleClient) - SparkEnv.set(mockEnv) - - val sem = new Semaphore(0) - var blockIds : Array[String] = null - when(shuffleClient.pushBlocks(any(), anyInt(), any(), any(), any())) - .thenAnswer((invocation: InvocationOnMock) => { - blockIds = invocation.getArguments()(2).asInstanceOf[Array[String]] - val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] - val blockFetchListener = - invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - (blockIds, managedBuffers).zipped.foreach((blockId, buffer) => { - blockFetchListener.onBlockFetchSuccess(blockId, buffer) - }) - sem.release() - }) - - writer.write(records) - writer.initiateBlockPush(blockResolver, writer.getPartitionLengths(), dependency, 0, 0, conf) - - sem.tryAcquire(1, 500, TimeUnit.MILLISECONDS) - for (i <- 1 to 3) { - assert(blockIds(i - 1) === ShuffleBlockId(0, 0, i).toString) - } - writer.stop(true) - } } From 0b951a762bb4b6b2c9001840a8cf93c4c89f7ce1 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 5 Nov 2020 21:56:02 -0800 Subject: [PATCH 14/35] Made the code compile, added more tests, introduced ShuffleBlockPushId --- .../org/apache/spark/storage/BlockId.scala | 5 + .../spark/shuffle/ShuffleWriterSuite.scala | 180 +++++++++++++----- 2 files changed, 139 insertions(+), 46 deletions(-) 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 3f69d5fc49853..c0a5dfb651ded 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -87,6 +87,11 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId } +@DeveloperApi +case class ShufflePushBlockId(shuffleId: Int, mapIndex: Integer, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapIndex + "_" + reduceId +} + @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala index 8fbf76334efa0..a115b402fe744 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala @@ -17,23 +17,24 @@ package org.apache.spark.shuffle +import java.io.File import java.net.ConnectException import java.nio.ByteBuffer -import java.util.concurrent.ExecutorService import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark._ -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockPushException, BlockStoreClient} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ @@ -45,10 +46,12 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ - private val conf: SparkConf = new SparkConf(loadDefaults = false) + private var conf: SparkConf = _ + private var pushedBlocks = new ArrayBuffer[String] override def beforeEach(): Unit = { super.beforeEach() + conf = new SparkConf(loadDefaults = false) MockitoAnnotations.initMocks(this) when(dependency.partitioner).thenReturn(new HashPartitioner(8)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) @@ -63,86 +66,169 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { when(blockManager.blockStoreClient).thenReturn(shuffleClient) } - test("test block push") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) - val allBlocks = new ArrayBuffer[String] + override def afterEach(): Unit = { + pushedBlocks.clear() + super.afterEach() + } + private def interceptPushedBlocksForSuccess(): Unit = { when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) .thenAnswer((invocation: InvocationOnMock) => { - val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - allBlocks ++= pushedBlocks + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - (pushedBlocks, managedBuffers).zipped.foreach((blockId, buffer) => { + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { blockFetchListener.onBlockFetchSuccess(blockId, buffer) }) }) + } + + test("Basic block push") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + interceptPushedBlocksForSuccess() + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("Large blocks are skipped for push") { + conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array(2, 2, 2, 2, 2, 2, 2, 1100)) + interceptPushedBlocksForSuccess() testWriter.initiateBlockPush( blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) + testWriter.stop(true) + } + + test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + interceptPushedBlocksForSuccess() + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(8)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + var blockPendingResponse : String = null + var listener : BlockFetchingListener = null + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + // Expecting 2 blocks + assert(blocks.length == 2) + if (blockPendingResponse == null) { + blockPendingResponse = blocks(1) + listener = blockFetchListener + // Respond with success only for the first block which will cause all the rest of the + // blocks to be deferred + blockFetchListener.onBlockFetchSuccess(blocks(0), managedBuffers(0)) + } else { + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + } + }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) - assert(allBlocks.length == dependency.partitioner.numPartitions) + assert(pushedBlocks.length == 2) + // this will trigger push of deferred blocks + listener.onBlockFetchSuccess(blockPendingResponse, mock(classOf[ManagedBuffer])) + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 8) testWriter.stop(true) } - test("error retries") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + test("Number of shuffle blocks grouped in a single push request is limited by " + + "maxBlockBatchSize") { + conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") + interceptPushedBlocksForSuccess() + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }) + testWriter.initiateBlockPush( + blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + testWriter.stop(true) + } + + test("Error retries") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) val errorHandler = testWriter.createErrorHandler() assert( !errorHandler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) assert( - errorHandler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + errorHandler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) assert (errorHandler.shouldRetryError(new Throwable())) } - test("error logging") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) + test("Error logging") { + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) val errorHandler = testWriter.createErrorHandler() assert( !errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) - assert( - !errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))) + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) + assert(!errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) assert(errorHandler.shouldLogError(new Throwable())) } - test("connect exceptions removes all the push requests for that host") { + test("Connect exceptions removes all the push requests for that host") { when(dependency.getMergerLocs).thenReturn( Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") - val executorService = mock(classOf[ExecutorService]) - when(executorService.submit(any[Runnable]())).thenAnswer(new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - } - }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions) - val allBlocks = new ArrayBuffer[String] - when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) .thenAnswer((invocation: InvocationOnMock) => { - val pushedBlocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - allBlocks ++= pushedBlocks + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - pushedBlocks.foreach(blockId => { + blocks.foreach(blockId => { blockFetchListener.onBlockFetchFailure( blockId, new RuntimeException(new ConnectException())) }) }) + val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, + Array.fill(dependency.partitioner.numPartitions) { 2 }) testWriter.initiateBlockPush( blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations - assert(allBlocks.length == 4) + assert(pushedBlocks.length == 4) } private class TestShuffleWriter( - private val numPartitions: Int) extends ShuffleWriter[Int, Int] { + private val numPartitions: Int, + private val partitionLengths: Array[Long]) extends ShuffleWriter[Int, Int] { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing @@ -158,16 +244,18 @@ class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } override def getPartitionLengths(): Array[Long] = { - Array.fill(numPartitions) { - 2 - } + partitionLengths } - override protected def sliceReqBufferIntoBlockBuffers( - reqBuffer: ManagedBuffer, blockSizes: Seq[Long]) = { - Array.fill(blockSizes.length) { - new NioManagedBuffer(ByteBuffer.wrap(Array[Byte](2))) - } + override protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + val managedBuffer = mock(classOf[ManagedBuffer]) + val byteBuffer = new Array[Byte](length.toInt) + when(managedBuffer.nioByteBuffer()).thenReturn(ByteBuffer.wrap(byteBuffer)) + managedBuffer } } } From 7d16198580c966b97953ac26f36bfd7c424defa8 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 10 Nov 2020 12:44:08 -0800 Subject: [PATCH 15/35] Added Since tags all the newly introduced classes --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 5 ----- 1 file changed, 5 deletions(-) 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 c0a5dfb651ded..3f69d5fc49853 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -87,11 +87,6 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId } -@DeveloperApi -case class ShufflePushBlockId(shuffleId: Int, mapIndex: Integer, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapIndex + "_" + reduceId -} - @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) From 415c2d0008fa24b3651df7c222a347023a60edd8 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 17 Nov 2020 18:56:38 -0800 Subject: [PATCH 16/35] Moved push of data out of ShuffleWriter and addressed other minor comments --- .../spark/shuffle/ShuffleWriterSuite.scala | 261 ------------------ 1 file changed, 261 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala deleted file mode 100644 index a115b402fe744..0000000000000 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleWriterSuite.scala +++ /dev/null @@ -1,261 +0,0 @@ -/* - * 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 java.io.File -import java.net.ConnectException -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer - -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark._ -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} -import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler -import org.apache.spark.network.util.TransportConf -import org.apache.spark.scheduler.MapStatus -import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.storage._ - -class ShuffleWriterSuite extends SparkFunSuite with BeforeAndAfterEach { - - @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ - @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ - @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ - - private var conf: SparkConf = _ - private var pushedBlocks = new ArrayBuffer[String] - - override def beforeEach(): Unit = { - super.beforeEach() - conf = new SparkConf(loadDefaults = false) - MockitoAnnotations.initMocks(this) - when(dependency.partitioner).thenReturn(new HashPartitioner(8)) - when(dependency.serializer).thenReturn(new JavaSerializer(conf)) - when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) - conf.set("spark.shuffle.push.based.enabled", "true") - conf.set("spark.shuffle.service.enabled", "true") - // Set the env because the shuffler writer gets the shuffle client instance from the env. - val mockEnv = mock(classOf[SparkEnv]) - when(mockEnv.conf).thenReturn(conf) - when(mockEnv.blockManager).thenReturn(blockManager) - SparkEnv.set(mockEnv) - when(blockManager.blockStoreClient).thenReturn(shuffleClient) - } - - override def afterEach(): Unit = { - pushedBlocks.clear() - super.afterEach() - } - - private def interceptPushedBlocksForSuccess(): Unit = { - when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) - .thenAnswer((invocation: InvocationOnMock) => { - val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - pushedBlocks ++= blocks - val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] - val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { - blockFetchListener.onBlockFetchSuccess(blockId, buffer) - }) - }) - } - - test("Basic block push") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - interceptPushedBlocksForSuccess() - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(1)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) - } - - test("Large blocks are skipped for push") { - conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array(2, 2, 2, 2, 2, 2, 2, 1100)) - interceptPushedBlocksForSuccess() - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(1)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) - testWriter.stop(true) - } - - test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { - conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") - interceptPushedBlocksForSuccess() - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(8)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) - } - - test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { - conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") - var blockPendingResponse : String = null - var listener : BlockFetchingListener = null - when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) - .thenAnswer((invocation: InvocationOnMock) => { - val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - pushedBlocks ++= blocks - val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] - val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - // Expecting 2 blocks - assert(blocks.length == 2) - if (blockPendingResponse == null) { - blockPendingResponse = blocks(1) - listener = blockFetchListener - // Respond with success only for the first block which will cause all the rest of the - // blocks to be deferred - blockFetchListener.onBlockFetchSuccess(blocks(0), managedBuffers(0)) - } else { - (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { - blockFetchListener.onBlockFetchSuccess(blockId, buffer) - }) - } - }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(1)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == 2) - // this will trigger push of deferred blocks - listener.onBlockFetchSuccess(blockPendingResponse, mock(classOf[ManagedBuffer])) - verify(shuffleClient, times(4)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == 8) - testWriter.stop(true) - } - - test("Number of shuffle blocks grouped in a single push request is limited by " + - "maxBlockBatchSize") { - conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") - interceptPushedBlocksForSuccess() - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(4)) - .pushBlocks(any(), any(), any(), any(), any()) - assert(pushedBlocks.length == dependency.partitioner.numPartitions) - testWriter.stop(true) - } - - test("Error retries") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - val errorHandler = testWriter.createErrorHandler() - assert( - !errorHandler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) - assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) - assert( - errorHandler.shouldRetryError(new RuntimeException(new IllegalArgumentException( - BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) - assert (errorHandler.shouldRetryError(new Throwable())) - } - - test("Error logging") { - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - val errorHandler = testWriter.createErrorHandler() - assert( - !errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) - assert(!errorHandler.shouldLogError(new RuntimeException( - new IllegalArgumentException( - BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) - assert(errorHandler.shouldLogError(new Throwable())) - } - - test("Connect exceptions removes all the push requests for that host") { - when(dependency.getMergerLocs).thenReturn( - Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) - conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") - when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) - .thenAnswer((invocation: InvocationOnMock) => { - val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] - pushedBlocks ++= blocks - val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - blocks.foreach(blockId => { - blockFetchListener.onBlockFetchFailure( - blockId, new RuntimeException(new ConnectException())) - }) - }) - val testWriter = new TestShuffleWriter(dependency.partitioner.numPartitions, - Array.fill(dependency.partitioner.numPartitions) { 2 }) - testWriter.initiateBlockPush( - blockResolver, testWriter.getPartitionLengths(), dependency, 0, 0, conf) - verify(shuffleClient, times(2)) - .pushBlocks(any(), any(), any(), any(), any()) - // 2 blocks for each merger locations - assert(pushedBlocks.length == 4) - } - - private class TestShuffleWriter( - private val numPartitions: Int, - private val partitionLengths: Array[Long]) extends ShuffleWriter[Int, Int] { - - override protected def submitTask(task: Runnable): Unit = { - // Making this synchronous for testing - task.run() - } - - /** Write a sequence of records to this task's output */ - override def write(records: Iterator[Product2[Int, Int]]): Unit = {} - - /** Close this writer, passing along whether the map completed */ - override def stop(success: Boolean): Option[MapStatus] = { - Option.empty - } - - override def getPartitionLengths(): Array[Long] = { - partitionLengths - } - - override protected def createRequestBuffer( - conf: TransportConf, - dataFile: File, - offset: Long, - length: Long): ManagedBuffer = { - val managedBuffer = mock(classOf[ManagedBuffer]) - val byteBuffer = new Array[Byte](length.toInt) - when(managedBuffer.nioByteBuffer()).thenReturn(ByteBuffer.wrap(byteBuffer)) - managedBuffer - } - } -} From 28d80984e265f6a9f2b6d29e24a32b161b8a8b25 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 17 Nov 2020 22:55:45 -0800 Subject: [PATCH 17/35] Renamed PushShuffleSupport to PushShufleComponent and changed initiate push to be triggered explictly --- .../org/apache/spark/executor/Executor.scala | 4 +- ...pport.scala => PushShuffleComponent.scala} | 10 ++--- .../spark/shuffle/ShuffleWriteProcessor.scala | 4 +- ....scala => PushShuffleComponentSuite.scala} | 39 +++++++++++-------- 4 files changed, 30 insertions(+), 27 deletions(-) rename core/src/main/scala/org/apache/spark/shuffle/{PushShuffleSupport.scala => PushShuffleComponent.scala} (99%) rename core/src/test/scala/org/apache/spark/shuffle/{PushShuffleSupportSuite.scala => PushShuffleComponentSuite.scala} (90%) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index dc5b4937d89b1..b2231ff476567 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.{FetchFailedException, PushShuffleSupport} +import org.apache.spark.shuffle.{FetchFailedException, PushShuffleComponent} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,7 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } - PushShuffleSupport.stop() + PushShuffleComponent.stop() threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly diff --git a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala rename to core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala index 7b0e302cd7843..4b810f80520cd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleSupport.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala @@ -36,7 +36,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf -import org.apache.spark.shuffle.PushShuffleSupport._ +import org.apache.spark.shuffle.PushShuffleComponent._ import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} import org.apache.spark.util.{ThreadUtils, Utils} @@ -53,7 +53,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} * @param partitionId map index of the shuffle map task * @param conf spark configuration */ -private[spark] class PushShuffleSupport( +private[spark] class PushShuffleComponent( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], @@ -70,8 +70,6 @@ private[spark] class PushShuffleSupport( private[this] val errorHandler = createErrorHandler() private[this] val unreachableBlockMgrs = new HashSet[BlockManagerId]() - initiateBlockPush() - // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { new BlockPushErrorHandler() { @@ -85,7 +83,7 @@ private[spark] class PushShuffleSupport( } } - def initiateBlockPush(): Unit = { + private[shuffle] def initiateBlockPush(): Unit = { val numPartitions = dep.partitioner.numPartitions val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") @@ -418,7 +416,7 @@ private[spark] class PushShuffleSupport( } } -private[spark] object PushShuffleSupport { +private[spark] object PushShuffleComponent { /** * A request to push blocks to a remote shuffle service diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index b88db3b320713..741c48d61bad2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -67,8 +67,8 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { if (Utils.isPushShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { val dataFile = manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver] .getDataFile(dep.shuffleId, mapId) - new PushShuffleSupport(dataFile, writer.getPartitionLengths(), dep, - partition.index, SparkEnv.get.conf) + new PushShuffleComponent(dataFile, writer.getPartitionLengths(), dep, + partition.index, SparkEnv.get.conf).initiateBlockPush() } } mapStatus.get diff --git a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala similarity index 90% rename from core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala index e859810f79f4e..590683662092c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleSupportSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ -class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { +class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @@ -84,34 +84,36 @@ class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { test("Basic block push") { interceptPushedBlocksForSuccess() - new TestPushShuffleSupport(mock(classOf[File]), + new TestPushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + .initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleSupport.stop() + PushShuffleComponent.stop() } test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") interceptPushedBlocksForSuccess() - new TestPushShuffleSupport(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), - dependency, 0, conf) + new TestPushShuffleComponent(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), + dependency, 0, conf).initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) - PushShuffleSupport.stop() + PushShuffleComponent.stop() } test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - new TestPushShuffleSupport(mock(classOf[File]), + new TestPushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + .initiateBlockPush() verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleSupport.stop() + PushShuffleComponent.stop() } test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { @@ -138,8 +140,9 @@ class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { }) } }) - new TestPushShuffleSupport(mock(classOf[File]), + new TestPushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + .initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 2) @@ -148,23 +151,24 @@ class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 8) - PushShuffleSupport.stop() + PushShuffleComponent.stop() } test("Number of shuffle blocks grouped in a single push request is limited by " + "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - new TestPushShuffleSupport(mock(classOf[File]), + new TestPushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0, conf) + .initiateBlockPush() verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleSupport.stop() + PushShuffleComponent.stop() } test("Error retries") { - val pushShuffleSupport = new PushShuffleSupport(mock(classOf[File]), + val pushShuffleSupport = new PushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -178,7 +182,7 @@ class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Error logging") { - val pushShuffleSupport = new PushShuffleSupport(mock(classOf[File]), + val pushShuffleSupport = new PushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -204,21 +208,22 @@ class PushShuffleSupportSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - new TestPushShuffleSupport(mock(classOf[File]), + new TestPushShuffleComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) + .initiateBlockPush() verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations assert(pushedBlocks.length == 4) } - private class TestPushShuffleSupport( + private class TestPushShuffleComponent( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], partitionId: Int, conf: SparkConf) - extends PushShuffleSupport(dataFile, partitionLengths, dep, partitionId, conf) { + extends PushShuffleComponent(dataFile, partitionLengths, dep, partitionId, conf) { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing From 5b725a23fd07190b0ae64ca9b18091c3337a013e Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 19 Nov 2020 10:13:00 -0800 Subject: [PATCH 18/35] Renamed PushShuffleComponent to PushShuffleWriterComponent --- .../org/apache/spark/executor/Executor.scala | 4 +-- .../spark/internal/config/package.scala | 6 ++-- ...scala => PushShuffleWriterComponent.scala} | 14 ++++---- .../spark/shuffle/ShuffleWriteProcessor.scala | 4 +-- ... => PushShuffleWriterComponentSuite.scala} | 32 +++++++++---------- 5 files changed, 30 insertions(+), 30 deletions(-) rename core/src/main/scala/org/apache/spark/shuffle/{PushShuffleComponent.scala => PushShuffleWriterComponent.scala} (97%) rename core/src/test/scala/org/apache/spark/shuffle/{PushShuffleComponentSuite.scala => PushShuffleWriterComponentSuite.scala} (91%) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b2231ff476567..aa5d37c5ec888 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.{FetchFailedException, PushShuffleComponent} +import org.apache.spark.shuffle.{FetchFailedException, PushShuffleWriterComponent} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,7 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly 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 63d31ac58d4b7..dea6f82726660 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 @@ -2024,7 +2024,7 @@ package object config { .doubleConf .createWithDefault(5) - private[spark] val PUSH_SHUFFLE_NUM_PUSH_THREADS = + private[spark] val SHUFFLE_NUM_PUSH_THREADS = ConfigBuilder("spark.shuffle.push.numPushThreads") .doc("Specify the number of threads in the block pusher pool. These threads assist " + "in creating connections and pushing blocks to remote shuffle services when push based " + @@ -2033,7 +2033,7 @@ package object config { .intConf .createOptional - private[spark] val PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = + private[spark] val SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockSizeToPush") .doc("The max size of an individual block to push to the remote shuffle services when push " + "based shuffle is enabled. Blocks larger than this threshold are not pushed.") @@ -2041,7 +2041,7 @@ package object config { .bytesConf(ByteUnit.KiB) .createWithDefaultString("800k") - private[spark] val PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE = + private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") .doc("The max size of a batch of shuffle blocks to be grouped into a single push request " + "when push based shuffle is enabled.") diff --git a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala rename to core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala index 4b810f80520cd..aa522788bad7f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleComponent.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala @@ -36,7 +36,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf -import org.apache.spark.shuffle.PushShuffleComponent._ +import org.apache.spark.shuffle.PushShuffleWriterComponent._ import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} import org.apache.spark.util.{ThreadUtils, Utils} @@ -53,7 +53,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} * @param partitionId map index of the shuffle map task * @param conf spark configuration */ -private[spark] class PushShuffleComponent( +private[spark] class PushShuffleWriterComponent( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], @@ -87,8 +87,8 @@ private[spark] class PushShuffleComponent( val numPartitions = dep.partitioner.numPartitions val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - val maxBlockSizeToPush = conf.get(PUSH_SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - val maxBlockBatchSize = conf.get(PUSH_SHUFFLE_MAX_BLOCK_BATCH_SIZE) * 1024 * 1024 + val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) @@ -416,7 +416,7 @@ private[spark] class PushShuffleComponent( } } -private[spark] object PushShuffleComponent { +private[spark] object PushShuffleWriterComponent { /** * A request to push blocks to a remote shuffle service @@ -443,8 +443,8 @@ private[spark] object PushShuffleComponent { private val BLOCK_PUSHER_POOL: ExecutorService = { val conf = SparkEnv.get.conf - if (Utils.isPushShuffleEnabled(conf)) { - val numThreads = conf.get(PUSH_SHUFFLE_NUM_PUSH_THREADS) + if (Utils.isPushBasedShuffleEnabled(conf)) { + val numThreads = conf.get(SHUFFLE_NUM_PUSH_THREADS) .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") } else { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 741c48d61bad2..52f2537d6dbbf 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -64,10 +64,10 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { // The map task only takes care of converting the shuffle data file into multiple // block push requests. It delegates pushing the blocks to a different thread-pool - // PushShuffleSupport.BLOCK_PUSHER_POOL. - if (Utils.isPushShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { + if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { val dataFile = manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver] .getDataFile(dep.shuffleId, mapId) - new PushShuffleComponent(dataFile, writer.getPartitionLengths(), dep, + new PushShuffleWriterComponent(dataFile, writer.getPartitionLengths(), dep, partition.index, SparkEnv.get.conf).initiateBlockPush() } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala similarity index 91% rename from core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala index 590683662092c..50b71653ee5a9 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleComponentSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ -class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { +class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @@ -84,36 +84,36 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { test("Basic block push") { interceptPushedBlocksForSuccess() - new TestPushShuffleComponent(mock(classOf[File]), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() } test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") interceptPushedBlocksForSuccess() - new TestPushShuffleComponent(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), dependency, 0, conf).initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() } test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - new TestPushShuffleComponent(mock(classOf[File]), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() } test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { @@ -140,7 +140,7 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { }) } }) - new TestPushShuffleComponent(mock(classOf[File]), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(1)) @@ -151,24 +151,24 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 8) - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() } test("Number of shuffle blocks grouped in a single push request is limited by " + "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - new TestPushShuffleComponent(mock(classOf[File]), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleComponent.stop() + PushShuffleWriterComponent.stop() } test("Error retries") { - val pushShuffleSupport = new PushShuffleComponent(mock(classOf[File]), + val pushShuffleSupport = new PushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -182,7 +182,7 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Error logging") { - val pushShuffleSupport = new PushShuffleComponent(mock(classOf[File]), + val pushShuffleSupport = new PushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -208,7 +208,7 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - new TestPushShuffleComponent(mock(classOf[File]), + new TestPushShuffleWriterComponent(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(2)) @@ -217,13 +217,13 @@ class PushShuffleComponentSuite extends SparkFunSuite with BeforeAndAfterEach { assert(pushedBlocks.length == 4) } - private class TestPushShuffleComponent( + private class TestPushShuffleWriterComponent( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], partitionId: Int, conf: SparkConf) - extends PushShuffleComponent(dataFile, partitionLengths, dep, partitionId, conf) { + extends PushShuffleWriterComponent(dataFile, partitionLengths, dep, partitionId, conf) { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing From a2f6635bc8b73b0f22c60ac30c84205bd4ad6c2e Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Sat, 21 Nov 2020 11:42:09 -0800 Subject: [PATCH 19/35] Not changing the default value of connectionCreationTimeout --- .../java/org/apache/spark/network/util/TransportConf.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 68a75a616a1ff..a840c8040f51d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -19,6 +19,7 @@ import java.util.Locale; import java.util.Properties; +import java.util.concurrent.TimeUnit; import com.google.common.primitives.Ints; import io.netty.util.NettyRuntime; @@ -107,8 +108,9 @@ public int connectionTimeoutMs() { /** Connect creation timeout in milliseconds. Default 30 secs. */ public int connectionCreationTimeoutMs() { + long connectionTimeoutS = TimeUnit.MILLISECONDS.toSeconds(connectionTimeoutMs()); long defaultTimeoutMs = JavaUtils.timeStringAsSec( - conf.get(SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY, "30s")) * 1000; + conf.get(SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY, connectionTimeoutS + "s")) * 1000; return (int) defaultTimeoutMs; } From 4a2aef76e098cc3224623a9670b23d054e4192d0 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 23 Nov 2020 15:27:48 -0800 Subject: [PATCH 20/35] Renamed PushShuffleWriterComponent to ShuffleBlockPusher and addressed other comments --- .../org/apache/spark/executor/Executor.scala | 4 +-- .../spark/internal/config/package.scala | 12 +++---- ...mponent.scala => ShuffleBlockPusher.scala} | 20 +++++------ .../spark/shuffle/ShuffleWriteProcessor.scala | 13 ++++--- ...te.scala => ShuffleBlockPusherSuite.scala} | 34 +++++++++---------- 5 files changed, 42 insertions(+), 41 deletions(-) rename core/src/main/scala/org/apache/spark/shuffle/{PushShuffleWriterComponent.scala => ShuffleBlockPusher.scala} (97%) rename core/src/test/scala/org/apache/spark/shuffle/{PushShuffleWriterComponentSuite.scala => ShuffleBlockPusherSuite.scala} (90%) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index aa5d37c5ec888..2bec61dca8944 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.{FetchFailedException, PushShuffleWriterComponent} +import org.apache.spark.shuffle.{FetchFailedException, ShuffleBlockPusher} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,7 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly 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 dea6f82726660..757e545912562 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 @@ -2027,24 +2027,24 @@ package object config { private[spark] val SHUFFLE_NUM_PUSH_THREADS = ConfigBuilder("spark.shuffle.push.numPushThreads") .doc("Specify the number of threads in the block pusher pool. These threads assist " + - "in creating connections and pushing blocks to remote shuffle services when push based " + - "shuffle is enabled. By default, the threadpool size is equal to the number of cores.") + "in creating connections and pushing blocks to remote shuffle services. By default, the " + + "threadpool size is equal to the number of cores.") .version("3.1.0") .intConf .createOptional private[spark] val SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockSizeToPush") - .doc("The max size of an individual block to push to the remote shuffle services when push " + - "based shuffle is enabled. Blocks larger than this threshold are not pushed.") + .doc("The max size of an individual block to push to the remote shuffle services. Blocks " + + "larger than this threshold are not pushed to be merged remotely. These shuffle blocks " + + "will be fetched by the executors in the original manner.") .version("3.1.0") .bytesConf(ByteUnit.KiB) .createWithDefaultString("800k") private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") - .doc("The max size of a batch of shuffle blocks to be grouped into a single push request " + - "when push based shuffle is enabled.") + .doc("The max size of a batch of shuffle blocks to be grouped into a single push request.") .version("3.1.0") .bytesConf(ByteUnit.MiB) // 2m is also the default value for TransportConf#memoryMapBytes. diff --git a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala rename to core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index aa522788bad7f..0af829e7ef1ba 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/PushShuffleWriterComponent.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -36,7 +36,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf -import org.apache.spark.shuffle.PushShuffleWriterComponent._ +import org.apache.spark.shuffle.ShuffleBlockPusher._ import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} import org.apache.spark.util.{ThreadUtils, Utils} @@ -53,7 +53,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} * @param partitionId map index of the shuffle map task * @param conf spark configuration */ -private[spark] class PushShuffleWriterComponent( +@Since("3.1.0") +private[spark] class ShuffleBlockPusher( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], @@ -89,8 +90,7 @@ private[spark] class PushShuffleWriterComponent( val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 - val mergerLocs = dep.getMergerLocs.map(loc => - BlockManagerId("", loc.host, loc.port)) + val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) @@ -416,7 +416,7 @@ private[spark] class PushShuffleWriterComponent( } } -private[spark] object PushShuffleWriterComponent { +private[spark] object ShuffleBlockPusher { /** * A request to push blocks to a remote shuffle service @@ -425,8 +425,7 @@ private[spark] object PushShuffleWriterComponent { * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous * blocks represented in this request */ - @Since("3.1.0") - case class PushRequest( + private case class PushRequest( address: BlockManagerId, blocks: Seq[(BlockId, Long)], reqBuffer: ManagedBuffer) { @@ -438,7 +437,6 @@ private[spark] object PushShuffleWriterComponent { * @param blockId blockId * @param failure exception if the push was unsuccessful; null otherwise; */ - @Since("3.1.0") private case class PushResult(blockId: String, failure: Throwable) private val BLOCK_PUSHER_POOL: ExecutorService = { @@ -446,16 +444,16 @@ private[spark] object PushShuffleWriterComponent { if (Utils.isPushBasedShuffleEnabled(conf)) { val numThreads = conf.get(SHUFFLE_NUM_PUSH_THREADS) .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) - ThreadUtils.newDaemonFixedThreadPool(numThreads, "block-push-thread") + ThreadUtils.newDaemonFixedThreadPool(numThreads, "shuffle-block-push-thread") } else { null } } /** - * Stop the shuffle writer pool if it isn't null. + * Stop the shuffle pusher pool if it isn't null. */ - private[spark] def stop() : Unit = { + private[spark] def stop(): Unit = { if (BLOCK_PUSHER_POOL != null) { BLOCK_PUSHER_POOL.shutdown() } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 52f2537d6dbbf..2162561343766 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -63,12 +63,15 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { // Initiate shuffle push process if push based shuffle is enabled // The map task only takes care of converting the shuffle data file into multiple // block push requests. It delegates pushing the blocks to a different thread-pool - - // PushShuffleSupport.BLOCK_PUSHER_POOL. + // ShuffleBlockPusher.BLOCK_PUSHER_POOL. if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { - val dataFile = manager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver] - .getDataFile(dep.shuffleId, mapId) - new PushShuffleWriterComponent(dataFile, writer.getPartitionLengths(), dep, - partition.index, SparkEnv.get.conf).initiateBlockPush() + manager.shuffleBlockResolver match { + case resolver: IndexShuffleBlockResolver => + val dataFile = resolver.getDataFile(dep.shuffleId, mapId) + new ShuffleBlockPusher(dataFile, writer.getPartitionLengths(), dep, + partition.index, SparkEnv.get.conf).initiateBlockPush() + case _ => + } } } mapStatus.get diff --git a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala similarity index 90% rename from core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index 50b71653ee5a9..bd77850407d15 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/PushShuffleWriterComponentSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ -class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterEach { +class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @@ -84,36 +84,36 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE test("Basic block push") { interceptPushedBlocksForSuccess() - new TestPushShuffleWriterComponent(mock(classOf[File]), + new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() } test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") interceptPushedBlocksForSuccess() - new TestPushShuffleWriterComponent(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), + new TestShuffleBlockPusher(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), dependency, 0, conf).initiateBlockPush() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() } test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - new TestPushShuffleWriterComponent(mock(classOf[File]), + new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() } test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { @@ -140,7 +140,7 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE }) } }) - new TestPushShuffleWriterComponent(mock(classOf[File]), + new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(1)) @@ -151,24 +151,24 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 8) - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() } test("Number of shuffle blocks grouped in a single push request is limited by " + "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - new TestPushShuffleWriterComponent(mock(classOf[File]), + new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) - PushShuffleWriterComponent.stop() + ShuffleBlockPusher.stop() } test("Error retries") { - val pushShuffleSupport = new PushShuffleWriterComponent(mock(classOf[File]), + val pushShuffleSupport = new ShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -182,7 +182,7 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE } test("Error logging") { - val pushShuffleSupport = new PushShuffleWriterComponent(mock(classOf[File]), + val pushShuffleSupport = new ShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) val errorHandler = pushShuffleSupport.createErrorHandler() assert( @@ -194,7 +194,7 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE assert(errorHandler.shouldLogError(new Throwable())) } - test("Connect exceptions removes all the push requests for that host") { + test("Connect exceptions remove all the push requests for that host") { when(dependency.getMergerLocs).thenReturn( Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") @@ -208,7 +208,7 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE blockId, new RuntimeException(new ConnectException())) }) }) - new TestPushShuffleWriterComponent(mock(classOf[File]), + new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) .initiateBlockPush() verify(shuffleClient, times(2)) @@ -217,13 +217,13 @@ class PushShuffleWriterComponentSuite extends SparkFunSuite with BeforeAndAfterE assert(pushedBlocks.length == 4) } - private class TestPushShuffleWriterComponent( + private class TestShuffleBlockPusher( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], partitionId: Int, conf: SparkConf) - extends PushShuffleWriterComponent(dataFile, partitionLengths, dep, partitionId, conf) { + extends ShuffleBlockPusher(dataFile, partitionLengths, dep, partitionId, conf) { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing From 1e9fb08926779e4f5982a750a61a01d3414368b2 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 25 Nov 2020 09:33:41 -0800 Subject: [PATCH 21/35] Addressed review comments --- .../org/apache/spark/shuffle/ShuffleBlockPusher.scala | 5 +++-- .../apache/spark/shuffle/ShuffleBlockPusherSuite.scala | 9 +++------ 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 0af829e7ef1ba..986793615a2d2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -69,7 +69,8 @@ private[spark] class ShuffleBlockPusher( private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() private[this] val pushRequests = new Queue[PushRequest] private[this] val errorHandler = createErrorHandler() - private[this] val unreachableBlockMgrs = new HashSet[BlockManagerId]() + // VisibleForTesting + private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { @@ -300,7 +301,7 @@ private[spark] class ShuffleBlockPusher( removed += droppedReq.get.length } logWarning(s"Received a ConnectException from $address. " + - s"Dropping push of $removed blocks and " + + s"Dropping $removed push-requests and " + s"not pushing any more blocks to this address.") } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index bd77850407d15..6fe0a3a47c2a2 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -208,13 +208,14 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - new TestShuffleBlockPusher(mock(classOf[File]), + val shuffleBlockPusher = new TestShuffleBlockPusher(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - .initiateBlockPush() + shuffleBlockPusher.initiateBlockPush() verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations assert(pushedBlocks.length == 4) + assert(shuffleBlockPusher.unreachableBlockMgrs.size == 2) } private class TestShuffleBlockPusher( @@ -230,10 +231,6 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { task.run() } - def getPartitionLengths(): Array[Long] = { - partitionLengths - } - override protected def createRequestBuffer( conf: TransportConf, dataFile: File, From 462af7adff5272d9c4b8f916e4a7ac2cc01d0326 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 25 Nov 2020 11:57:36 -0800 Subject: [PATCH 22/35] Fixed the documentation for numPushThreads --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 757e545912562..2a60cca5e2bb9 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 @@ -2028,7 +2028,7 @@ package object config { ConfigBuilder("spark.shuffle.push.numPushThreads") .doc("Specify the number of threads in the block pusher pool. These threads assist " + "in creating connections and pushing blocks to remote shuffle services. By default, the " + - "threadpool size is equal to the number of cores.") + "threadpool size is equal to the number of spark executor cores.") .version("3.1.0") .intConf .createOptional From a88ffd6114ea7ee232406e0f19f53f5279043705 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 30 Nov 2020 23:22:43 -0800 Subject: [PATCH 23/35] Changed the initiateBlockPush method, added tests, and stop pushing completely when too late --- .../spark/shuffle/ShuffleBlockPusher.scala | 82 +++++----- .../spark/shuffle/ShuffleWriteProcessor.scala | 4 +- .../shuffle/ShuffleBlockPusherSuite.scala | 147 ++++++++++++++---- 3 files changed, 163 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 986793615a2d2..e36c0c5e61908 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -45,21 +45,13 @@ import org.apache.spark.util.{ThreadUtils, Utils} * When push shuffle is enabled, it is created after the shuffle writer finishes writing the shuffle * file and initiates the block push process. * - * @param dataFile mapper generated shuffle data file - * @param partitionLengths array of shuffle block size so we can tell shuffle block - * boundaries within the shuffle file - * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle - * services to push local shuffle blocks - * @param partitionId map index of the shuffle map task - * @param conf spark configuration + * @param conf spark configuration */ @Since("3.1.0") private[spark] class ShuffleBlockPusher( - dataFile: File, - partitionLengths: Array[Long], - dep: ShuffleDependency[_, _, _], - partitionId: Int, conf: SparkConf) extends Logging { + private[this] var maxBlockSizeToPush = 0L + private[this] var maxBlockBatchSize = 0L private[this] var maxBytesInFlight = 0L private[this] var maxReqsInFlight = 0 private[this] var maxBlocksInFlightPerAddress = 0 @@ -71,6 +63,17 @@ private[spark] class ShuffleBlockPusher( private[this] val errorHandler = createErrorHandler() // VisibleForTesting private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() + private[this] var stopPushing = false + + initialize() + + private def initialize(): Unit = { + maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 + maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) + } // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { @@ -85,20 +88,26 @@ private[spark] class ShuffleBlockPusher( } } - private[shuffle] def initiateBlockPush(): Unit = { + /** + * Initiates the block push. + * + * @param dataFile mapper generated shuffle data file + * @param partitionLengths array of shuffle block size so we can tell shuffle block + * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle + * services to push local shuffle blocks + * @param partitionId map index of the shuffle map task + */ + private[shuffle] def initiateBlockPush( + dataFile: File, + partitionLengths: Array[Long], + dep: ShuffleDependency[_, _, _], + partitionId: Int + ): Unit = { val numPartitions = dep.partitioner.numPartitions - val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - - val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) - - maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 - maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) - maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) - + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, - partitionLengths, mergerLocs, transportConf, maxBlockSizeToPush, maxBlockBatchSize) + partitionLengths, mergerLocs, transportConf) // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same // time won't be pushing the same ranges of shuffle partitions. pushRequests ++= Utils.randomize(requests) @@ -133,7 +142,7 @@ private[spark] class ShuffleBlockPusher( if (deferredPushRequests.nonEmpty) { for ((remoteAddress, defReqQueue) <- deferredPushRequests) { while (isRemoteBlockPushable(defReqQueue) && - !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front) && !stopPushing) { val request = defReqQueue.dequeue() logDebug(s"Processing deferred push request for $remoteAddress with " + s"${request.blocks.length} blocks") @@ -146,7 +155,7 @@ private[spark] class ShuffleBlockPusher( } // Process any regular push requests if possible. - while (isRemoteBlockPushable(pushRequests)) { + while (isRemoteBlockPushable(pushRequests) && !stopPushing) { val request = pushRequests.dequeue() val remoteAddress = request.address if (isRemoteAddressMaxedOut(remoteAddress, request)) { @@ -307,6 +316,7 @@ private[spark] class ShuffleBlockPusher( } if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") + stopPushing = true return false } else { remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) @@ -322,29 +332,25 @@ private[spark] class ShuffleBlockPusher( * manner to make sure each target location receives shuffle blocks belonging to the same set * of partition ranges. 0-length blocks and blocks that are large enough will be skipped. * - * @param numPartitions Number of shuffle partitions in the shuffle file + * @param numPartitions sumber of shuffle partitions in the shuffle file * @param partitionId map index of the current mapper - * @param shuffleId ShuffleId of current shuffle - * @param dataFile Shuffle data file + * @param shuffleId shuffleId of current shuffle + * @param dataFile shuffle data file * @param partitionLengths array of sizes of blocks in the shuffle data file - * @param mergerLocs Target locations to push blocks to - * @param transportConf TransportConf used to create FileSegmentManagedBuffer - * @param maxBlockSizeToPush Max size of individual blocks that will be pushed. Blocks larger - * than this threshold will be skipped. - * @param maxBlockBatchSize Max size of a batch of shuffle blocks to be grouped into a single - * request + * @param mergerLocs target locations to push blocks to + * @param transportConf transportConf used to create FileSegmentManagedBuffer * @return List of the PushRequest, randomly shuffled. + * + * VisibleForTesting */ - private def prepareBlockPushRequests( + private[shuffle] def prepareBlockPushRequests( numPartitions: Int, partitionId: Int, shuffleId: Int, dataFile: File, partitionLengths: Array[Long], mergerLocs: Seq[BlockManagerId], - transportConf: TransportConf, - maxBlockSizeToPush: Long, - maxBlockBatchSize: Long): Seq[PushRequest] = { + transportConf: TransportConf): Seq[PushRequest] = { var offset = 0L var currentReqSize = 0L var currentReqOffset = 0L @@ -426,7 +432,7 @@ private[spark] object ShuffleBlockPusher { * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous * blocks represented in this request */ - private case class PushRequest( + private[spark] case class PushRequest( address: BlockManagerId, blocks: Seq[(BlockId, Long)], reqBuffer: ManagedBuffer) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 2162561343766..abff650b0611b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -68,8 +68,8 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { manager.shuffleBlockResolver match { case resolver: IndexShuffleBlockResolver => val dataFile = resolver.getDataFile(dep.shuffleId, mapId) - new ShuffleBlockPusher(dataFile, writer.getPartitionLengths(), dep, - partition.index, SparkEnv.get.conf).initiateBlockPush() + new ShuffleBlockPusher(SparkEnv.get.conf) + .initiateBlockPush(dataFile, writer.getPartitionLengths(), dep, partition.index) case _ => } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index 6fe0a3a47c2a2..d0d48d47578e7 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler import org.apache.spark.network.util.TransportConf import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.ShuffleBlockPusher.PushRequest import org.apache.spark.storage._ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -82,11 +83,51 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { }) } + private def verifyPushRequests( + pushRequests: Seq[PushRequest], + expectedSizes: Seq[Int]): Unit = { + (pushRequests, expectedSizes).zipped.foreach((req, size) => { + assert(req.size == size) + }) + } + + test("A batch of blocks is limited by maxBlocksBatchSize") { + conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") + conf.set("spark.shuffle.push.maxBlockSizeToPush", "2048k") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val largeBlockSize = 2 * 1024 * 1024 + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, largeBlockSize, largeBlockSize), mergerLocs, + mock(classOf[TransportConf])) + assert(pushRequests.length == 3) + verifyPushRequests(pushRequests, Seq(6, largeBlockSize, largeBlockSize)) + } + + test("Large blocks are excluded in the preparation") { + conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, 1028, 1024), mergerLocs, mock(classOf[TransportConf])) + assert(pushRequests.length == 2) + verifyPushRequests(pushRequests, Seq(6, 1024)) + } + + test("Number of blocks in a push request are limited by maxBlocksInFlightPerAddress ") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, 2, 2), mergerLocs, mock(classOf[TransportConf])) + assert(pushRequests.length == 5) + verifyPushRequests(pushRequests, Seq(2, 2, 2, 2, 2)) + } + test("Basic block push") { interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - .initiateBlockPush() + new TestShuffleBlockPusher(conf).initiateBlockPush(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -96,8 +137,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), - dependency, 0, conf).initiateBlockPush() + new TestShuffleBlockPusher(conf).initiateBlockPush( + mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), dependency, 0) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) @@ -107,9 +148,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - .initiateBlockPush() + new TestShuffleBlockPusher(conf).initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -140,9 +180,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { }) } }) - new TestShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - .initiateBlockPush() + new TestShuffleBlockPusher(conf).initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 2) @@ -158,9 +197,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0, conf) - .initiateBlockPush() + new TestShuffleBlockPusher(conf).initiateBlockPush(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0) verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -168,9 +206,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Error retries") { - val pushShuffleSupport = new ShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - val errorHandler = pushShuffleSupport.createErrorHandler() + val pusher = new ShuffleBlockPusher(conf) + val errorHandler = pusher.createErrorHandler() assert( !errorHandler.shouldRetryError(new RuntimeException( new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) @@ -182,9 +219,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { } test("Error logging") { - val pushShuffleSupport = new ShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - val errorHandler = pushShuffleSupport.createErrorHandler() + val pusher = new ShuffleBlockPusher(conf) + val errorHandler = pusher.createErrorHandler() assert( !errorHandler.shouldLogError(new RuntimeException( new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) @@ -194,6 +230,63 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { assert(errorHandler.shouldLogError(new Throwable())) } + test("Blocks are continued to push even when a block push fails with collision " + + "exception") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val pusher = new TestShuffleBlockPusher(conf) + var failBlock: Boolean = true + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + blocks.foreach(blockId => { + if (failBlock) { + failBlock = false + // Fail the first block with the collision exception. + blockFetchListener.onBlockFetchFailure(blockId, new RuntimeException( + new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX))) + } else { + pushedBlocks += blockId + blockFetchListener.onBlockFetchSuccess(blockId, mock(classOf[ManagedBuffer])) + } + }) + }) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + verify(shuffleClient, times(8)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 7) + } + + test("More blocks are not pushed when a block push fails with too late " + + "exception") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val pusher = new TestShuffleBlockPusher(conf) + var failBlock: Boolean = true + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + blocks.foreach(blockId => { + if (failBlock) { + failBlock = false + // Fail the first block with the collision exception. + blockFetchListener.onBlockFetchFailure(blockId, new RuntimeException( + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX))) + } else { + pushedBlocks += blockId + blockFetchListener.onBlockFetchSuccess(blockId, mock(classOf[ManagedBuffer])) + } + }) + }) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.isEmpty) + } + test("Connect exceptions remove all the push requests for that host") { when(dependency.getMergerLocs).thenReturn( Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) @@ -208,9 +301,9 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - val shuffleBlockPusher = new TestShuffleBlockPusher(mock(classOf[File]), - Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0, conf) - shuffleBlockPusher.initiateBlockPush() + val shuffleBlockPusher = new TestShuffleBlockPusher(conf) + shuffleBlockPusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations @@ -218,13 +311,7 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { assert(shuffleBlockPusher.unreachableBlockMgrs.size == 2) } - private class TestShuffleBlockPusher( - dataFile: File, - partitionLengths: Array[Long], - dep: ShuffleDependency[_, _, _], - partitionId: Int, - conf: SparkConf) - extends ShuffleBlockPusher(dataFile, partitionLengths, dep, partitionId, conf) { + private class TestShuffleBlockPusher(conf: SparkConf) extends ShuffleBlockPusher(conf) { override protected def submitTask(task: Runnable): Unit = { // Making this synchronous for testing From 08386f3549789a70c01cf0743117c4b0694963f3 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 2 Dec 2020 09:37:43 -0800 Subject: [PATCH 24/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala Co-authored-by: wuyi --- .../scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index e36c0c5e61908..1259b8fb0873b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -101,8 +101,7 @@ private[spark] class ShuffleBlockPusher( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], - partitionId: Int - ): Unit = { + partitionId: Int): Unit = { val numPartitions = dep.partitioner.numPartitions val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") @@ -466,4 +465,3 @@ private[spark] object ShuffleBlockPusher { } } } - From d5370dbf39d568f1476c81cb75313691bdf80258 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 2 Dec 2020 09:41:44 -0800 Subject: [PATCH 25/35] Addressed review comments --- .../spark/shuffle/ShuffleBlockPusher.scala | 25 ++++++------------- 1 file changed, 8 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 1259b8fb0873b..37e8cd9f03682 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -48,13 +48,14 @@ import org.apache.spark.util.{ThreadUtils, Utils} * @param conf spark configuration */ @Since("3.1.0") -private[spark] class ShuffleBlockPusher( - conf: SparkConf) extends Logging { - private[this] var maxBlockSizeToPush = 0L - private[this] var maxBlockBatchSize = 0L - private[this] var maxBytesInFlight = 0L - private[this] var maxReqsInFlight = 0 - private[this] var maxBlocksInFlightPerAddress = 0 +private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { + private[this] val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 + private[this] val maxBlockBatchSize = + conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 + private[this] val maxBytesInFlight = + conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + private[this] val maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + private[this] val maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) private[this] var bytesInFlight = 0L private[this] var reqsInFlight = 0 private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() @@ -65,16 +66,6 @@ private[spark] class ShuffleBlockPusher( private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() private[this] var stopPushing = false - initialize() - - private def initialize(): Unit = { - maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 - maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 - maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) - maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) - } - // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { new BlockPushErrorHandler() { From fd0e98b0c2fc1bf1c38bae6499743f5a1c7463b5 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 3 Dec 2020 10:20:02 -0800 Subject: [PATCH 26/35] Addressed review comments --- .../apache/spark/internal/config/package.scala | 11 ++++++----- .../spark/shuffle/ShuffleBlockPusher.scala | 18 ++++++++---------- .../shuffle/ShuffleBlockPusherSuite.scala | 2 +- 3 files changed, 15 insertions(+), 16 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 2a60cca5e2bb9..19a5a396fbb3e 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 @@ -2039,16 +2039,17 @@ package object config { "larger than this threshold are not pushed to be merged remotely. These shuffle blocks " + "will be fetched by the executors in the original manner.") .version("3.1.0") - .bytesConf(ByteUnit.KiB) + .bytesConf(ByteUnit.BYTE) .createWithDefaultString("800k") private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") .doc("The max size of a batch of shuffle blocks to be grouped into a single push request.") .version("3.1.0") - .bytesConf(ByteUnit.MiB) - // 2m is also the default value for TransportConf#memoryMapBytes. - // Having this default to 2m will very likely make each batch of block loaded in memory with - // memory mapping, which has higher overhead with small MB sized chunk of data. + .bytesConf(ByteUnit.BYTE) + // 2m is also the default value for TransportConf#memoryMapBytes so it is better to have + // the batch size greater than 2m. If it defaults to 2m as well it is very likely that each + // batch of block will be loaded in memory with memory mapping, which has higher overhead + // with small MB sized chunk of data. .createWithDefaultString("3m") } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 37e8cd9f03682..266a72e09a894 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -49,9 +49,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} */ @Since("3.1.0") private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { - private[this] val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) * 1024 - private[this] val maxBlockBatchSize = - conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) * 1024 * 1024 + private[this] val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) + private[this] val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) private[this] val maxBytesInFlight = conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 private[this] val maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) @@ -86,18 +85,17 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { * @param partitionLengths array of shuffle block size so we can tell shuffle block * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle * services to push local shuffle blocks - * @param partitionId map index of the shuffle map task + * @param mapIndex map index of the shuffle map task */ private[shuffle] def initiateBlockPush( dataFile: File, partitionLengths: Array[Long], dep: ShuffleDependency[_, _, _], - partitionId: Int): Unit = { + mapIndex: Int): Unit = { val numPartitions = dep.partitioner.numPartitions - val mergerLocs = dep.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - val requests = prepareBlockPushRequests(numPartitions, partitionId, dep.shuffleId, dataFile, - partitionLengths, mergerLocs, transportConf) + val requests = prepareBlockPushRequests(numPartitions, mapIndex, dep.shuffleId, dataFile, + partitionLengths, dep.getMergerLocs, transportConf) // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same // time won't be pushing the same ranges of shuffle partitions. pushRequests ++= Utils.randomize(requests) @@ -118,10 +116,10 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { } /** - * Since multiple netty client threads could potentially be calling pushUpToMax for the same + * Since multiple block push threads could potentially be calling pushUpToMax for the same * mapper, we synchronize access to this method so that only one thread can push blocks for * a given mapper. This helps to simplify access to the shared states. The down side of this - * is that we could unnecessarily block other mappers' block pushes if all netty client threads + * is that we could unnecessarily block other mappers' block pushes if all the threads * are occupied by block pushes from the same mapper. * * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index d0d48d47578e7..d7600a4166c9e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -271,7 +271,7 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { blocks.foreach(blockId => { if (failBlock) { failBlock = false - // Fail the first block with the collision exception. + // Fail the first block with the too late exception. blockFetchListener.onBlockFetchFailure(blockId, new RuntimeException( new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX))) } else { From 8c9a482f6b8ad8519232d14ff7c115b495615594 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 3 Dec 2020 21:16:49 -0800 Subject: [PATCH 27/35] Changed the default of maxBlockSizeToPush to 1M --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 19a5a396fbb3e..4268f7918bc1f 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 @@ -2040,7 +2040,7 @@ package object config { "will be fetched by the executors in the original manner.") .version("3.1.0") .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("800k") + .createWithDefaultString("1m") private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") From f8632b37efead4f271e8035846751f5d524c3bcb Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 3 Dec 2020 22:49:48 -0800 Subject: [PATCH 28/35] Removed the stopPushing flag which is not needed --- .../spark/shuffle/ShuffleBlockPusher.scala | 6 +-- .../shuffle/ShuffleBlockPusherSuite.scala | 41 ++++++++++++++----- 2 files changed, 33 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 266a72e09a894..8fe9e242b5b4b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -63,7 +63,6 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { private[this] val errorHandler = createErrorHandler() // VisibleForTesting private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() - private[this] var stopPushing = false // VisibleForTesting private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { @@ -130,7 +129,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { if (deferredPushRequests.nonEmpty) { for ((remoteAddress, defReqQueue) <- deferredPushRequests) { while (isRemoteBlockPushable(defReqQueue) && - !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front) && !stopPushing) { + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { val request = defReqQueue.dequeue() logDebug(s"Processing deferred push request for $remoteAddress with " + s"${request.blocks.length} blocks") @@ -143,7 +142,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { } // Process any regular push requests if possible. - while (isRemoteBlockPushable(pushRequests) && !stopPushing) { + while (isRemoteBlockPushable(pushRequests)) { val request = pushRequests.dequeue() val remoteAddress = request.address if (isRemoteAddressMaxedOut(remoteAddress, request)) { @@ -304,7 +303,6 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { } if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") - stopPushing = true return false } else { remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index d7600a4166c9e..011ff86658837 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle import java.io.File import java.net.ConnectException import java.nio.ByteBuffer +import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.ArrayBuffer @@ -126,8 +127,10 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { test("Basic block push") { interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(conf).initiateBlockPush(mock(classOf[File]), + val blockPusher = new TestShuffleBlockPusher(conf) + blockPusher.initiateBlockPush(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + blockPusher.runPendingTasks() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -137,8 +140,10 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { test("Large blocks are skipped for push") { conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(conf).initiateBlockPush( + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) @@ -148,8 +153,10 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(conf).initiateBlockPush( + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -180,13 +187,16 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { }) } }) - new TestShuffleBlockPusher(conf).initiateBlockPush( + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 2) // this will trigger push of deferred blocks listener.onBlockFetchSuccess(blockPendingResponse, mock(classOf[ManagedBuffer])) + pusher.runPendingTasks() verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 8) @@ -197,8 +207,10 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { "maxBlockBatchSize") { conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") interceptPushedBlocksForSuccess() - new TestShuffleBlockPusher(conf).initiateBlockPush(mock(classOf[File]), + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush(mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(4)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == dependency.partitioner.numPartitions) @@ -254,6 +266,7 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { }) pusher.initiateBlockPush( mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(8)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.length == 7) @@ -282,6 +295,7 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { }) pusher.initiateBlockPush( mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(1)) .pushBlocks(any(), any(), any(), any(), any()) assert(pushedBlocks.isEmpty) @@ -301,21 +315,28 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { blockId, new RuntimeException(new ConnectException())) }) }) - val shuffleBlockPusher = new TestShuffleBlockPusher(conf) - shuffleBlockPusher.initiateBlockPush( + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() verify(shuffleClient, times(2)) .pushBlocks(any(), any(), any(), any(), any()) // 2 blocks for each merger locations assert(pushedBlocks.length == 4) - assert(shuffleBlockPusher.unreachableBlockMgrs.size == 2) + assert(pusher.unreachableBlockMgrs.size == 2) } private class TestShuffleBlockPusher(conf: SparkConf) extends ShuffleBlockPusher(conf) { + private[this] val tasks = new LinkedBlockingQueue[Runnable] override protected def submitTask(task: Runnable): Unit = { - // Making this synchronous for testing - task.run() + tasks.add(task) + } + + def runPendingTasks(): Unit = { + while (!tasks.isEmpty) { + tasks.take().run() + } } override protected def createRequestBuffer( From 6d0fadea97b4bcbf5769fcf0aa674c7981fd66cb Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Fri, 4 Dec 2020 09:31:10 -0800 Subject: [PATCH 29/35] Addressed review comments --- .../main/scala/org/apache/spark/internal/config/package.scala | 4 ++-- .../org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala | 2 ++ 2 files changed, 4 insertions(+), 2 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 4268f7918bc1f..093bd8de40896 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 @@ -2047,8 +2047,8 @@ package object config { .doc("The max size of a batch of shuffle blocks to be grouped into a single push request.") .version("3.1.0") .bytesConf(ByteUnit.BYTE) - // 2m is also the default value for TransportConf#memoryMapBytes so it is better to have - // the batch size greater than 2m. If it defaults to 2m as well it is very likely that each + // Default is 3m because it is greater than 2m which is the default value for + // TransportConf#memoryMapBytes. If this defaults to 2m as well it is very likely that each // batch of block will be loaded in memory with memory mapping, which has higher overhead // with small MB sized chunk of data. .createWithDefaultString("3m") diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala index 011ff86658837..cc561e6106019 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -334,6 +334,8 @@ class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { } def runPendingTasks(): Unit = { + // This ensures that all the submitted tasks - updateStateAndCheckIfPushMore and pushUpToMax + // are run synchronously. while (!tasks.isEmpty) { tasks.take().run() } From 7d70f8200165b34057a11e07c708833d7ee7412a Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 16 Dec 2020 19:09:46 -0800 Subject: [PATCH 30/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala Co-authored-by: Mridul Muralidharan <1591700+mridulm@users.noreply.github.com> --- .../scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 8fe9e242b5b4b..e737db4e15df8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -292,10 +292,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { var removed = 0 unreachableBlockMgrs.add(address) removed += pushRequests.dequeueAll(req => req.address == address).length - val droppedReq = deferredPushRequests.remove(address) - if (droppedReq.isDefined) { - removed += droppedReq.get.length - } + removed += deferredPushRequests.remove(address).map(_.length).getOrElse(0) logWarning(s"Received a ConnectException from $address. " + s"Dropping $removed push-requests and " + s"not pushing any more blocks to this address.") From 23cc5028e8d1f01c7fe51983060699ff3cab744c Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 16 Dec 2020 19:10:43 -0800 Subject: [PATCH 31/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala Co-authored-by: Mridul Muralidharan <1591700+mridulm@users.noreply.github.com> --- .../scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index e737db4e15df8..f3b8192ccb11a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -147,9 +147,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { val remoteAddress = request.address if (isRemoteAddressMaxedOut(remoteAddress, request)) { logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") - val defReqQueue = deferredPushRequests.getOrElse(remoteAddress, new Queue[PushRequest]()) - defReqQueue.enqueue(request) - deferredPushRequests(remoteAddress) = defReqQueue + deferredPushRequests.getOrElseUpdate(remoteAddress, new Queue[PushRequest]()).enqueue(request) } else { sendRequest(request) } From bcebb13eaa1da1d6970c6f99b2c6996e1f6fbd4d Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 16 Dec 2020 19:38:21 -0800 Subject: [PATCH 32/35] Addressed Mridul's comments --- .../apache/spark/internal/config/package.scala | 6 +++--- .../spark/shuffle/ShuffleBlockPusher.scala | 18 +++++++++--------- .../org/apache/spark/storage/BlockId.scala | 2 +- 3 files changed, 13 insertions(+), 13 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 093bd8de40896..f0e8aed66184c 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 @@ -2029,7 +2029,7 @@ package object config { .doc("Specify the number of threads in the block pusher pool. These threads assist " + "in creating connections and pushing blocks to remote shuffle services. By default, the " + "threadpool size is equal to the number of spark executor cores.") - .version("3.1.0") + .version("3.2.0") .intConf .createOptional @@ -2038,14 +2038,14 @@ package object config { .doc("The max size of an individual block to push to the remote shuffle services. Blocks " + "larger than this threshold are not pushed to be merged remotely. These shuffle blocks " + "will be fetched by the executors in the original manner.") - .version("3.1.0") + .version("3.2.0") .bytesConf(ByteUnit.BYTE) .createWithDefaultString("1m") private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") .doc("The max size of a batch of shuffle blocks to be grouped into a single push request.") - .version("3.1.0") + .version("3.2.0") .bytesConf(ByteUnit.BYTE) // Default is 3m because it is greater than 2m which is the default value for // TransportConf#memoryMapBytes. If this defaults to 2m as well it is very likely that each diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index f3b8192ccb11a..63cf510eb9e5a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -47,7 +47,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} * * @param conf spark configuration */ -@Since("3.1.0") +@Since("3.2.0") private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { private[this] val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) private[this] val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) @@ -147,7 +147,8 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { val remoteAddress = request.address if (isRemoteAddressMaxedOut(remoteAddress, request)) { logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") - deferredPushRequests.getOrElseUpdate(remoteAddress, new Queue[PushRequest]()).enqueue(request) + deferredPushRequests.getOrElseUpdate(remoteAddress, new Queue[PushRequest]()) + .enqueue(request) } else { sendRequest(request) } @@ -176,8 +177,8 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { * pushes. */ private def sendRequest(request: PushRequest): Unit = { - bytesInFlight = bytesInFlight + request.size - reqsInFlight = reqsInFlight + 1 + bytesInFlight += request.size + reqsInFlight += 1 numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( request.address, 0) + request.blocks.length @@ -276,13 +277,12 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { remainingBlocks: HashSet[String], pushResult: PushResult): Boolean = synchronized { remainingBlocks -= pushResult.blockId - bytesInFlight = bytesInFlight - bytesPushed + bytesInFlight -= bytesPushed numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 if (remainingBlocks.isEmpty) { - reqsInFlight = reqsInFlight - 1 + reqsInFlight -= 1 } - if (pushResult.failure != null && pushResult.failure.getCause != null && - pushResult.failure.getCause.isInstanceOf[ConnectException]) { + if (pushResult.failure != null && pushResult.failure.getCause.isInstanceOf[ConnectException]) { // Remove all the blocks for this address just once because removing from pushRequests // is expensive. If there is a ConnectException for the first block, all the subsequent // blocks to that address will fail, so should avoid removing multiple times. @@ -363,6 +363,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { // Convert the previous batch into a PushRequest requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + blocks = new ArrayBuffer[(BlockId, Long)] } // Start a new batch currentReqSize = 0 @@ -370,7 +371,6 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { // of currentReqOffset and when we are about to start a new batch currentReqOffset = -1 currentMergerId = mergerId - blocks = new ArrayBuffer[(BlockId, Long)] } // Only push blocks under the size limit if (blockSize <= maxBlockSizeToPush) { 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 3f69d5fc49853..73bf809a08a68 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -81,7 +81,7 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } -@Since("3.1.0") +@Since("3.2.0") @DeveloperApi case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) extends BlockId { override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId From 762ac1eb3b77b6b9468518d70eba1f4961fabf49 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Fri, 18 Dec 2020 18:17:28 -0800 Subject: [PATCH 33/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala Co-authored-by: Min Shen --- .../scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 63cf510eb9e5a..241356797d186 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -255,8 +255,8 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { blockOffsets.zip(blockSizes).map { case (offset, size) => new NioManagedBuffer(inMemoryBuffer.duplicate() - .position(offset.toInt) - .limit((offset + size).toInt).asInstanceOf[ByteBuffer].slice()) + .position(offset) + .limit(offset + size.toInt).asInstanceOf[ByteBuffer].slice()) }.toArray } } From 6aae02a8b5e2cdbb6d42b783e18e97d1f81fa6cd Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Fri, 18 Dec 2020 18:17:36 -0800 Subject: [PATCH 34/35] Update core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala Co-authored-by: Min Shen --- .../scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 241356797d186..8ba917bb3d4b3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -246,8 +246,8 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { Array(reqBuffer) } else { val inMemoryBuffer = reqBuffer.nioByteBuffer() - val blockOffsets = new Array[Long](blockSizes.size) - var offset = 0L + val blockOffsets = new Array[Int](blockSizes.size) + var offset = 0 for (index <- blockSizes.indices) { blockOffsets(index) = offset offset += blockSizes(index) From 21ea881f398a35038e2359ef0b4ed5cd15aab736 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Fri, 18 Dec 2020 20:20:47 -0800 Subject: [PATCH 35/35] Coverting blockSize to Int during preparation of push request --- .../spark/shuffle/ShuffleBlockPusher.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 8ba917bb3d4b3..88d084ce1b2f4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -241,7 +241,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { */ private def sliceReqBufferIntoBlockBuffers( reqBuffer: ManagedBuffer, - blockSizes: Seq[Long]): Array[ManagedBuffer] = { + blockSizes: Seq[Int]): Array[ManagedBuffer] = { if (blockSizes.size == 1) { Array(reqBuffer) } else { @@ -256,7 +256,7 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { case (offset, size) => new NioManagedBuffer(inMemoryBuffer.duplicate() .position(offset) - .limit(offset + size.toInt).asInstanceOf[ByteBuffer].slice()) + .limit(offset + size).asInstanceOf[ByteBuffer].slice()) }.toArray } } @@ -333,12 +333,12 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { mergerLocs: Seq[BlockManagerId], transportConf: TransportConf): Seq[PushRequest] = { var offset = 0L - var currentReqSize = 0L + var currentReqSize = 0 var currentReqOffset = 0L var currentMergerId = 0 val numMergers = mergerLocs.length val requests = new ArrayBuffer[PushRequest] - var blocks = new ArrayBuffer[(BlockId, Long)] + var blocks = new ArrayBuffer[(BlockId, Int)] for (reduceId <- 0 until numPartitions) { val blockSize = partitionLengths(reduceId) logDebug( @@ -357,13 +357,13 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { && blocks.size < maxBlocksInFlightPerAddress && mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { // Add current block to current batch - currentReqSize += blockSize + currentReqSize += blockSize.toInt } else { if (blocks.nonEmpty) { // Convert the previous batch into a PushRequest requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) - blocks = new ArrayBuffer[(BlockId, Long)] + blocks = new ArrayBuffer[(BlockId, Int)] } // Start a new batch currentReqSize = 0 @@ -374,13 +374,14 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { } // Only push blocks under the size limit if (blockSize <= maxBlockSizeToPush) { - blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSize)) + val blockSizeInt = blockSize.toInt + blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSizeInt)) // Only update currentReqOffset if the current block is the first in the request if (currentReqOffset == -1) { currentReqOffset = offset } if (currentReqSize == 0) { - currentReqSize += blockSize + currentReqSize += blockSizeInt } } } @@ -415,7 +416,7 @@ private[spark] object ShuffleBlockPusher { */ private[spark] case class PushRequest( address: BlockManagerId, - blocks: Seq[(BlockId, Long)], + blocks: Seq[(BlockId, Int)], reqBuffer: ManagedBuffer) { val size = blocks.map(_._2).sum }