From ed2f7d418b9d713bd304d5edb28f687e30b3cb60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Thu, 18 Apr 2019 15:00:10 +0200 Subject: [PATCH 01/25] Initial version --- .../network/shuffle/BlockTransferClient.java | 31 ++++ .../shuffle/ExternalShuffleBlockHandler.java | 60 +++++-- .../shuffle/ExternalShuffleBlockResolver.java | 19 ++ .../spark/network/shuffle/ShuffleClient.java | 2 +- .../ExternalShuffleBlockHandlerSuite.java | 46 +++-- .../spark/ExecutorAllocationManager.scala | 4 +- .../network/BlockTransferClientSync.scala | 65 +++++++ .../spark/network/BlockTransferService.scala | 46 +---- .../apache/spark/storage/BlockManager.scala | 43 +++-- .../spark/storage/BlockManagerMaster.scala | 8 +- .../storage/BlockManagerMasterEndpoint.scala | 85 +++++++-- .../spark/storage/BlockManagerMessages.scala | 3 +- .../apache/spark/storage/StorageUtils.scala | 19 +- .../org/apache/spark/DistributedSuite.scala | 10 +- .../spark/ExternalShuffleServiceSuite.scala | 49 ++++- .../network/BlockTransferServiceSuite.scala | 5 +- .../spark/storage/BlockManagerInfoSuite.scala | 167 ++++++++++++++++++ .../spark/storage/BlockManagerSuite.scala | 82 ++++++--- 18 files changed, 598 insertions(+), 146 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java create mode 100644 core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java new file mode 100644 index 000000000000..30f6e6f97035 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java @@ -0,0 +1,31 @@ +/* + * 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.network.shuffle; + +public interface BlockTransferClient { + + /** + * Fetch a sequence of blocks from a remote node asynchronously. + */ + void fetchBlocks( + String host, + int port, + String execId, + String[] blockIds, + BlockFetchingListener listener, + DownloadFileManager downloadFileManager); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 70dcc8b8b8b6..42382b9df107 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.function.Function; import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; @@ -48,9 +49,9 @@ /** * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. * - * Handles registering executors and opening shuffle blocks from them. Shuffle blocks are registered - * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- - * level shuffle block. + * Handles registering executors and opening shuffle or disk persisted RDD blocks from them. + * Blocks are registered with the "one-for-one" strategy, meaning each Transport-layer Chunk + * is equivalent to one block. */ public class ExternalShuffleBlockHandler extends RpcHandler { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); @@ -213,21 +214,42 @@ public Map getMetrics() { private class ManagedBufferIterator implements Iterator { private int index = 0; - private final String appId; - private final String execId; - private final int shuffleId; - // An array containing mapId and reduceId pairs. - private final int[] mapIdAndReduceIds; - - ManagedBufferIterator(String appId, String execId, String[] blockIds) { - this.appId = appId; - this.execId = execId; + private final Function blockDataForIndexFn; + private final int size; + + ManagedBufferIterator(final String appId, final String execId, String[] blockIds) { String[] blockId0Parts = blockIds[0].split("_"); - if (blockId0Parts.length != 4 || !blockId0Parts[0].equals("shuffle")) { - throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[0]); + if (blockId0Parts.length == 4 && blockId0Parts[0].equals("shuffle")) { + final int shuffleId = Integer.parseInt(blockId0Parts[1]); + final int[] mapIdAndReduceIds = shuffleMapIdAndReduceIds(blockIds, shuffleId); + size = mapIdAndReduceIds.length; + blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, shuffleId, + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + } else if(blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { + final int[] rddAndSplitIds = rddAndSplitIds(blockIds); + size = rddAndSplitIds.length; + blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, + rddAndSplitIds[index], rddAndSplitIds[index + 1]); + } else { + throw new IllegalArgumentException("Unexpected block id format: " + blockIds[0]); } - this.shuffleId = Integer.parseInt(blockId0Parts[1]); - mapIdAndReduceIds = new int[2 * blockIds.length]; + } + + private int[] rddAndSplitIds(String[] blockIds) { + final int[] rddAndSplitIds = new int[2 * blockIds.length]; + for (int i = 0; i < blockIds.length; i++) { + String[] blockIdParts = blockIds[i].split("_"); + if (blockIdParts.length != 3 || !blockIdParts[0].equals("rdd")) { + throw new IllegalArgumentException("Unexpected RDD block id format: " + blockIds[i]); + } + rddAndSplitIds[2 * i] = Integer.parseInt(blockIdParts[1]); + rddAndSplitIds[2 * i + 1] = Integer.parseInt(blockIdParts[2]); + } + return rddAndSplitIds; + } + + private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { + final int[] mapIdAndReduceIds = new int[2 * blockIds.length]; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { @@ -240,17 +262,17 @@ private class ManagedBufferIterator implements Iterator { mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } + return mapIdAndReduceIds; } @Override public boolean hasNext() { - return index < mapIdAndReduceIds.length; + return index < size; } @Override public ManagedBuffer next() { - final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + final ManagedBuffer block = blockDataForIndexFn.apply(index); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 0b7a27402369..70cb115278ca 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -179,6 +179,18 @@ public ManagedBuffer getBlockData( return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); } + public ManagedBuffer getBlockData( + String appId, + String execId, + int rddId, + int splitIndex) { + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + return getDiskPersistedRddBlockData(executor, rddId, splitIndex); + } /** * Removes our metadata of all executors registered for the given application, and optionally * also deletes the local directories associated with the executors of that application in a @@ -298,6 +310,13 @@ private ManagedBuffer getSortBasedShuffleBlockData( } } + public ManagedBuffer getDiskPersistedRddBlockData( + ExecutorShuffleInfo executor, int rddId, int splitIndex) { + File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, + "rdd_" + rddId + "_" + splitIndex); + return new FileSegmentManagedBuffer(conf, file, 0, file.length()); + } + /** * Hashes a filename into the corresponding local directory, in a manner consistent with * Spark's DiskBlockManager.getFile(). diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index 62b99c40f61f..cea637d066fa 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -23,7 +23,7 @@ import com.codahale.metrics.MetricSet; /** Provides an interface for reading shuffle files, either from an Executor or external service. */ -public abstract class ShuffleClient implements Closeable { +public abstract class ShuffleClient implements BlockTransferClient, Closeable { /** * Initializes the ShuffleClient, specifying this Executor's appId. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 537c277cd26b..c48cb6f92941 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -49,6 +49,10 @@ public class ExternalShuffleBlockHandlerSuite { OneForOneStreamManager streamManager; ExternalShuffleBlockResolver blockResolver; RpcHandler handler; + ManagedBuffer blockMarkers[] = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), + new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) + }; @Before public void beforeEach() { @@ -76,20 +80,35 @@ public void testRegisterExecutor() { assertEquals(1, registerExecutorRequestLatencyMillis.getCount()); } - @SuppressWarnings("unchecked") @Test public void testOpenShuffleBlocks() { + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); + + checkOpenBlocksReceive(new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }); + + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + verifyOpenBlockLatencyMetrics(); + } + + @Test + public void testOpenDiskPersistedRDDBlocks() { + when(blockResolver.getBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getBlockData("app0", "exec1", 0, 1)).thenReturn(blockMarkers[1]); + + checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }); + + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 1); + verifyOpenBlockLatencyMetrics(); + } + + private void checkOpenBlocksReceive(String[] blockIds) { when(client.getClientId()).thenReturn("app0"); RpcResponseCallback callback = mock(RpcResponseCallback.class); - - ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); - ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(block1Marker); - ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) - .toByteBuffer(); + ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", blockIds).toByteBuffer(); handler.receive(client, openBlocks, callback); ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); @@ -106,13 +125,12 @@ public void testOpenShuffleBlocks() { verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), any()); Iterator buffers = stream.getValue(); - assertEquals(block0Marker, buffers.next()); - assertEquals(block1Marker, buffers.next()); + assertEquals(blockMarkers[0], buffers.next()); + assertEquals(blockMarkers[1], buffers.next()); assertFalse(buffers.hasNext()); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + } - // Verify open block request latency metrics + private void verifyOpenBlockLatencyMetrics() { Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) .getAllMetrics() .getMetrics() diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 6fade10b7a3c..952f216c13c1 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -492,7 +492,7 @@ private[spark] class ExecutorAllocationManager( if (testing || executorsRemoved.nonEmpty) { executorsRemoved.foreach { removedExecutorId => // If it is a cached block, it uses cachedExecutorIdleTimeoutS for timeout - val idleTimeout = if (blockManagerMaster.hasCachedBlocks(removedExecutorId)) { + val idleTimeout = if (blockManagerMaster.hasExclusiveCachedBlocks(removedExecutorId)) { cachedExecutorIdleTimeoutS } else { executorIdleTimeoutS @@ -608,7 +608,7 @@ private[spark] class ExecutorAllocationManager( // Note that it is not necessary to query the executors since all the cached // blocks we are concerned with are reported to the driver. Note that this // does not include broadcast blocks. - val hasCachedBlocks = blockManagerMaster.hasCachedBlocks(executorId) + val hasCachedBlocks = blockManagerMaster.hasExclusiveCachedBlocks(executorId) val now = clock.getTimeMillis() val timeout = { if (hasCachedBlocks) { diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala new file mode 100644 index 000000000000..65225fb781bf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala @@ -0,0 +1,65 @@ +/* + * 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.network + +import java.nio.ByteBuffer + +import scala.concurrent.Promise +import scala.concurrent.duration.Duration + +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockTransferClient, DownloadFileManager} +import org.apache.spark.storage.EncryptedManagedBuffer +import org.apache.spark.util.ThreadUtils + +private[spark] class BlockTransferClientSync(val blockTransferClient: BlockTransferClient) { + + /** + * A special case of fetchBlocks, as it fetches only one block and is blocking. + * + * It is also only available after the BlockTransferClient is initialised. + */ + def fetchBlockSync(host: String, port: Int, execId: String, blockId: String, + tempFileManager: DownloadFileManager): ManagedBuffer = { + // A monitor for the thread to wait on. + val result = Promise[ManagedBuffer]() + blockTransferClient.fetchBlocks(host, port, execId, Array(blockId), + new BlockFetchingListener { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + result.failure(exception) + } + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + data match { + case f: FileSegmentManagedBuffer => + result.success(f) + case e: EncryptedManagedBuffer => + result.success(e) + case _ => + try { + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result.success(new NioManagedBuffer(ret)) + } catch { + case e: Throwable => result.failure(e) + } + } + } + }, tempFileManager) + ThreadUtils.awaitResult(result.future, Duration.Inf) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 51ced697c6af..c5f2f85512fd 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -18,16 +18,15 @@ package org.apache.spark.network import java.io.Closeable -import java.nio.ByteBuffer -import scala.concurrent.{Future, Promise} +import scala.concurrent.{Future} import scala.concurrent.duration.Duration import scala.reflect.ClassTag import org.apache.spark.internal.Logging -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} -import org.apache.spark.storage.{BlockId, EncryptedManagedBuffer, StorageLevel} +import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.ThreadUtils private[spark] @@ -82,45 +81,6 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo level: StorageLevel, classTag: ClassTag[_]): Future[Unit] - /** - * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. - * - * It is also only available after [[init]] is invoked. - */ - def fetchBlockSync( - host: String, - port: Int, - execId: String, - blockId: String, - tempFileManager: DownloadFileManager): ManagedBuffer = { - // A monitor for the thread to wait on. - val result = Promise[ManagedBuffer]() - fetchBlocks(host, port, execId, Array(blockId), - new BlockFetchingListener { - override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - result.failure(exception) - } - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - data match { - case f: FileSegmentManagedBuffer => - result.success(f) - case e: EncryptedManagedBuffer => - result.success(e) - case _ => - try { - val ret = ByteBuffer.allocate(data.size.toInt) - ret.put(data.nioByteBuffer()) - ret.flip() - result.success(new NioManagedBuffer(ret)) - } catch { - case e: Throwable => result.failure(e) - } - } - } - }, tempFileManager) - ThreadUtils.awaitResult(result.future, Duration.Inf) - } - /** * Upload a single block to a remote node, available only after [[init]] is invoked. * diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 09928e47634c..8baeb7ca1b08 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -138,6 +138,9 @@ private[spark] class BlockManager( private val remoteReadNioBufferConversion = conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION) + // Visible for testing + private[storage] var blockTransferClientSync = new BlockTransferClientSync(blockTransferService) + val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. val deleteFilesOnStop = @@ -164,20 +167,7 @@ private[spark] class BlockManager( private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory - // Port used by the external shuffle service. In Yarn mode, this may be already be - // set through the Hadoop configuration as the server is launched in the Yarn NM. - private val externalShuffleServicePort = { - val tmpPort = Utils.getSparkOrYarnConfig(conf, config.SHUFFLE_SERVICE_PORT.key, - config.SHUFFLE_SERVICE_PORT.defaultValueString).toInt - if (tmpPort == 0) { - // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds - // an open port. But we still need to tell our spark apps the right port to use. So - // only if the yarn config has the port set to 0, we prefer the value in the spark config - conf.get(config.SHUFFLE_SERVICE_PORT.key).toInt - } else { - tmpPort - } - } + private val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf) var blockManagerId: BlockManagerId = _ @@ -843,7 +833,7 @@ private[spark] class BlockManager( * * This does not acquire a lock on this block in this JVM. */ - private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { + private[spark] def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val ct = implicitly[ClassTag[T]] getRemoteManagedBuffer(blockId).map { data => val values = @@ -852,21 +842,30 @@ private[spark] class BlockManager( } } + private def preferExecutors(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { + val (executors, shuffleServers) = locations.partition(_.port != externalShuffleServicePort) + executors ++ shuffleServers + } + /** * Return a list of locations for the given block, prioritizing the local machine since * multiple block managers can share the same host, followed by hosts on the same rack. + * + * Within each of the above listed groups (same host, same rack and others) executors are + * preferred over the external shuffle service. */ - private def sortLocations(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { + private[spark] def sortLocations(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { val locs = Random.shuffle(locations) - val (preferredLocs, otherLocs) = locs.partition { loc => blockManagerId.host == loc.host } - blockManagerId.topologyInfo match { - case None => preferredLocs ++ otherLocs + val (preferredLocs, otherLocs) = locs.partition(_.host == blockManagerId.host) + val orderedParts = blockManagerId.topologyInfo match { + case None => Seq(preferredLocs, otherLocs) case Some(_) => val (sameRackLocs, differentRackLocs) = otherLocs.partition { loc => blockManagerId.topologyInfo == loc.topologyInfo } - preferredLocs ++ sameRackLocs ++ differentRackLocs + Seq(preferredLocs, sameRackLocs, differentRackLocs) } + orderedParts.map(preferExecutors).reduce(_ ++ _) } /** @@ -902,8 +901,8 @@ private[spark] class BlockManager( val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { - blockTransferService.fetchBlockSync( - loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager) + blockTransferClientSync.fetchBlockSync(loc.host, loc.port, loc.executorId, blockId.toString, + tempFileManager) } catch { case NonFatal(e) => runningFailureCount += 1 diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index d24421b96277..ab5b89d346a0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -223,11 +223,11 @@ class BlockManagerMaster( } /** - * Find out if the executor has cached blocks. This method does not consider broadcast blocks, - * since they are not reported the master. + * Find out if the executor has cached blocks which are only available via this executor. + * This method does not consider broadcast blocks, since they are not reported the master. */ - def hasCachedBlocks(executorId: String): Boolean = { - driverEndpoint.askSync[Boolean](HasCachedBlocks(executorId)) + def hasExclusiveCachedBlocks(executorId: String): Boolean = { + driverEndpoint.askSync[Boolean](HasExclusiveCachedBlocks(executorId)) } /** Stop the driver endpoint, called only on the Spark driver node */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index f388d59e78ba..0737926385e8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -48,6 +48,10 @@ class BlockManagerMasterEndpoint( // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] + // Mapping from external shuffle service block manager id to the block statuses. + private val blockStatusByShuffleService = + new mutable.HashMap[BlockManagerId, JHashMap[BlockId, BlockStatus]] + // Mapping from executor ID to block manager ID. private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] @@ -71,6 +75,8 @@ class BlockManagerMasterEndpoint( val proactivelyReplicate = conf.get(config.STORAGE_REPLICATION_PROACTIVE) logInfo("BlockManagerMasterEndpoint up") + private val externalShuffleServiceEnabled: Boolean = conf.get(config.SHUFFLE_SERVICE_ENABLED) + private val externalShuffleServicePort: Int = StorageUtils.externalShuffleServicePort(conf) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => @@ -135,12 +141,12 @@ class BlockManagerMasterEndpoint( case BlockManagerHeartbeat(blockManagerId) => context.reply(heartbeatReceived(blockManagerId)) - case HasCachedBlocks(executorId) => + case HasExclusiveCachedBlocks(executorId) => blockManagerIdByExecutor.get(executorId) match { case Some(bm) => if (blockManagerInfo.contains(bm)) { val bmInfo = blockManagerInfo(bm) - context.reply(bmInfo.cachedBlocks.nonEmpty) + context.reply(bmInfo.exclusiveCachedBlocks.nonEmpty) } else { context.reply(false) } @@ -353,6 +359,10 @@ class BlockManagerMasterEndpoint( ).map(_.flatten.toSeq) } + private def externalShuffleServiceIdOnHost(blockManagerId: BlockManagerId): BlockManagerId = { + BlockManagerId(blockManagerId.executorId, blockManagerId.host, externalShuffleServicePort) + } + /** * Returns the BlockManagerId with topology information populated, if available. */ @@ -384,8 +394,17 @@ class BlockManagerMasterEndpoint( blockManagerIdByExecutor(id.executorId) = id - blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) + val externalShuffleServiceBlockStatus = + if (externalShuffleServiceEnabled) { + val externalShuffleServiceBlocks = blockStatusByShuffleService + .getOrElseUpdate(externalShuffleServiceIdOnHost(id), new JHashMap[BlockId, BlockStatus]) + Some(externalShuffleServiceBlocks) + } else { + None + } + + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxOnHeapMemSize, + maxOffHeapMemSize, slaveEndpoint, externalShuffleServiceBlockStatus) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) @@ -430,6 +449,15 @@ class BlockManagerMasterEndpoint( locations.remove(blockManagerId) } + if (storageLevel.useDisk && externalShuffleServiceEnabled) { + val externalShuffleServiceId = externalShuffleServiceIdOnHost(blockManagerId) + if (storageLevel.isValid) { + locations.add(externalShuffleServiceId) + } else { + locations.remove(externalShuffleServiceId) + } + } + // Remove the block from master tracking if it has been removed on all slaves. if (locations.size == 0) { blockLocations.remove(blockId) @@ -443,7 +471,13 @@ class BlockManagerMasterEndpoint( private def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = { val locations = Option(blockLocations.get(blockId)).map(_.toSeq).getOrElse(Seq.empty) - val status = locations.headOption.flatMap { bmId => blockManagerInfo(bmId).getStatus(blockId) } + val status = locations.headOption.flatMap { bmId => + if (externalShuffleServiceEnabled && bmId.port == externalShuffleServicePort) { + Option(blockStatusByShuffleService(bmId).get(blockId)) + } else { + blockManagerInfo(bmId).getStatus(blockId) + } + } if (locations.nonEmpty && status.isDefined) { Some(BlockLocationsAndStatus(locations, status.get)) @@ -499,19 +533,25 @@ private[spark] class BlockManagerInfo( timeMs: Long, val maxOnHeapMem: Long, val maxOffHeapMem: Long, - val slaveEndpoint: RpcEndpointRef) + val slaveEndpoint: RpcEndpointRef, + val externalShuffleServiceBlockStatus: Option[JHashMap[BlockId, BlockStatus]]) extends Logging { val maxMem = maxOnHeapMem + maxOffHeapMem + val externalShuffleServiceEnabled = externalShuffleServiceBlockStatus.isDefined + private var _lastSeenMs: Long = timeMs private var _remainingMem: Long = maxMem // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - // Cached blocks held by this BlockManager. This does not include broadcast blocks. - private val _cachedBlocks = new mutable.HashSet[BlockId] + /** + * Cached blocks held exclusively by this BlockManager. This does not include broadcast blocks + * and local disc persisted blocks when external shuffle service is enabled. + */ + private val _exclusiveCachedBlocks = new mutable.HashSet[BlockId] def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId)) @@ -579,13 +619,28 @@ private[spark] class BlockManagerInfo( s" (size: ${Utils.bytesToString(diskSize)})") } } - if (!blockId.isBroadcast && blockStatus.isCached) { - _cachedBlocks += blockId + + if (!blockId.isBroadcast) { + if (!externalShuffleServiceEnabled || !storageLevel.useDisk) { + _exclusiveCachedBlocks += blockId + } else if (blockExists) { + // removing block from the exclusive cached blocks when updated to non-exclusive + _exclusiveCachedBlocks -= blockId + } + } + + externalShuffleServiceBlockStatus.foreach { shuffleServiceBlocks => + if (!blockId.isBroadcast && blockStatus.diskSize > 0) { + shuffleServiceBlocks.put(blockId, blockStatus) + } } } else if (blockExists) { // If isValid is not true, drop the block. _blocks.remove(blockId) - _cachedBlocks -= blockId + _exclusiveCachedBlocks -= blockId + externalShuffleServiceBlockStatus.foreach { blockStatus => + blockStatus.remove(blockId) + } if (originalLevel.useMemory) { logInfo(s"Removed $blockId on ${blockManagerId.hostPort} in memory" + s" (size: ${Utils.bytesToString(originalMemSize)}," + @@ -602,8 +657,11 @@ private[spark] class BlockManagerInfo( if (_blocks.containsKey(blockId)) { _remainingMem += _blocks.get(blockId).memSize _blocks.remove(blockId) + externalShuffleServiceBlockStatus.foreach { blockStatus => + blockStatus.remove(blockId) + } } - _cachedBlocks -= blockId + _exclusiveCachedBlocks -= blockId } def remainingMem: Long = _remainingMem @@ -612,8 +670,7 @@ private[spark] class BlockManagerInfo( def blocks: JHashMap[BlockId, BlockStatus] = _blocks - // This does not include broadcast blocks. - def cachedBlocks: collection.Set[BlockId] = _cachedBlocks + def exclusiveCachedBlocks: collection.Set[BlockId] = _exclusiveCachedBlocks override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 2be28420b495..3dbac694cf81 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -122,7 +122,8 @@ private[spark] object BlockManagerMessages { case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class HasCachedBlocks(executorId: String) extends ToBlockManagerMaster + case class HasExclusiveCachedBlocks(executorId: String) extends ToBlockManagerMaster case class IsExecutorAlive(executorId: String) extends ToBlockManagerMaster + } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 1c9ea1dba97d..fc426eee608c 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -26,7 +26,8 @@ import org.apache.commons.lang3.{JavaVersion, SystemUtils} import sun.misc.Unsafe import sun.nio.ch.DirectBuffer -import org.apache.spark.internal.Logging +import org.apache.spark.SparkConf +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.Utils /** @@ -236,4 +237,20 @@ private[spark] object StorageUtils extends Logging { } } + /** + * Get the port used by the external shuffle service. In Yarn mode, this may be already be + * set through the Hadoop configuration as the server is launched in the Yarn NM. + */ + def externalShuffleServicePort(conf: SparkConf): Int = { + val tmpPort = Utils.getSparkOrYarnConfig(conf, config.SHUFFLE_SERVICE_PORT.key, + config.SHUFFLE_SERVICE_PORT.defaultValueString).toInt + if (tmpPort == 0) { + // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds + // an open port. But we still need to tell our spark apps the right port to use. So + // only if the yarn config has the port set to 0, we prefer the value in the spark config + conf.get(config.SHUFFLE_SERVICE_PORT.key).toInt + } else { + tmpPort + } + } } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index aad20545bafb..4c96a0328eee 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests._ +import org.apache.spark.network.BlockTransferClientSync import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -185,13 +186,18 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager val blockTransfer = blockManager.blockTransferService + val blockTransferClientSync = new BlockTransferClientSync(blockTransfer) val serializerManager = SparkEnv.get.serializerManager val locations = blockManager.master.getLocations(blockId) assert(locations.size === storageLevel.replication, s"; got ${locations.size} replicas instead of ${storageLevel.replication}") locations.foreach { cmId => - val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, - blockId.toString, null) + val bytes = blockTransferClientSync.fetchBlockSync( + cmId.host, + cmId.port, + cmId.executorId, + blockId.toString, + null) val deserialized = serializerManager.dataDeserializeStream(blockId, new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 8b737cd8c81f..0908add2f008 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -18,12 +18,15 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ import org.apache.spark.internal.config import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.Utils /** @@ -32,7 +35,7 @@ import org.apache.spark.util.Utils * set up in `ExternalShuffleBlockHandler`, such as changing the format of shuffle files or how * we hash files into folders. */ -class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { +class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll with Eventually { var server: TransportServer = _ var transportContext: TransportContext = _ var rpcHandler: ExternalShuffleBlockHandler = _ @@ -92,4 +95,48 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { } e.getMessage should include ("Fetch failure will not retry stage due to testing config") } + + test("SPARK-25888: using external shuffle service fetching disk persisted blocks") { + sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) + sc.env.blockManager.externalShuffleServiceEnabled should equal(true) + sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) + + // In a slow machine, one slave may register hundreds of milliseconds ahead of the other one. + // If we don't wait for all slaves, it's possible that only one executor runs all jobs. Then + // all shuffle blocks will be in this executor, ShuffleBlockFetcherIterator will directly fetch + // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. + // In this case, we won't receive FetchFailed. And it will make this test fail. + // Therefore, we should wait until all slaves are up + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) + + val rdd = sc.parallelize(0 until 100, 2) + .map(i => (i, 1)) + .persist(StorageLevel.DISK_ONLY) + + rdd.count() + + val blockId = RDDBlockId(rdd.id, 0) + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 2) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + sc.killExecutors(sc.getExecutorIds()) + + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 1) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + val rddSplit0Block = sc.env.blockManager.getRemoteValues(blockId) + assert(rddSplit0Block.isDefined) + + // Invalidate the registered executors, disallowing access to their shuffle blocks (without + // deleting the actual shuffle files, so we could access them without the shuffle service). + rpcHandler.applicationRemoved(sc.conf.getAppId, false /* cleanupLocalDirs */) + } } diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala index d7e4b9166fa0..b603034c525a 100644 --- a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala @@ -93,10 +93,11 @@ class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits { } } + val blockTransferClientSync = new BlockTransferClientSync(blockTransferService) val e = intercept[SparkException] { failAfter(10.seconds) { - blockTransferService.fetchBlockSync( - "localhost-unused", 0, "exec-id-unused", "block-id-unused", null) + blockTransferClientSync.fetchBlockSync("localhost-unused", 0, "exec-id-unused", + "block-id-unused", null) } } assert(e.getCause.isInstanceOf[IllegalArgumentException]) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala new file mode 100644 index 000000000000..d0329fa29fde --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.util.{HashMap => JHashMap} + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite + +class BlockManagerInfoSuite extends SparkFunSuite { + + def testWithShuffleServiceOnOff(testName: String) + (f: (Boolean, BlockManagerInfo) => Unit): Unit = { + Seq(true, false).foreach { shuffleServiceEnabled => + val blockManagerInfo = new BlockManagerInfo( + BlockManagerId("executor0", "host", 1234, None), + timeMs = 300, + maxOnHeapMem = 10000, + maxOffHeapMem = 20000, + slaveEndpoint = null, + if (shuffleServiceEnabled) Some(new JHashMap[BlockId, BlockStatus]) else None) + test(s"$testName externalShuffleServiceEnabled=$shuffleServiceEnabled") { + f(shuffleServiceEnabled, blockManagerInfo) + } + } + } + + testWithShuffleServiceOnOff("add broadcast block") { (_, blockManagerInfo) => + val broadcastId: BlockId = BroadcastBlockId(0, "field1") + blockManagerInfo.updateBlockInfo( + broadcastId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 100) + assert(blockManagerInfo.blocks.asScala + === Map(broadcastId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 100))) + assert(blockManagerInfo.exclusiveCachedBlocks === Set()) + assert(blockManagerInfo.remainingMem == 29800) + } + + testWithShuffleServiceOnOff("add RDD block with MEMORY_ONLY") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, diskSize = 0) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(blockManagerInfo.exclusiveCachedBlocks === Set(rddId)) + assert(blockManagerInfo.remainingMem == 29800) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + } + } + + testWithShuffleServiceOnOff("add RDD block with MEMORY_AND_DISK") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo( + rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) + assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(blockManagerInfo.remainingMem == 29800) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala + === Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + } + } + + testWithShuffleServiceOnOff("add RDD block with DISK_ONLY") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) + assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + } + + testWithShuffleServiceOnOff("update RDD block from MEMORY_ONLY to DISK_ONLY") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(blockManagerInfo.exclusiveCachedBlocks === Set(rddId)) + assert(blockManagerInfo.remainingMem == 29800) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala == Map()) + } + + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForNoMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) + assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForNoMemoryOnly) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + } + + testWithShuffleServiceOnOff("using invalid StorageLevel") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) + assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.NONE, memSize = 0, diskSize = 200) + assert(blockManagerInfo.blocks.asScala === Map()) + assert(blockManagerInfo.exclusiveCachedBlocks === Set()) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + } + } + + testWithShuffleServiceOnOff("remove block") { + (shuffleServiceEnabled, blockManagerInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(blockManagerInfo.blocks.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) + assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala + === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + + blockManagerInfo.removeBlock(rddId) + assert(blockManagerInfo.blocks.asScala === Map()) + assert(blockManagerInfo.exclusiveCachedBlocks === Set()) + assert(blockManagerInfo.remainingMem == 30000) + if (shuffleServiceEnabled) { + assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9f3d8f291ede..44e34a390fda 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -39,12 +39,12 @@ import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager -import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} +import org.apache.spark.network.{BlockDataManager, BlockTransferClientSync, BlockTransferService, TransportContext} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} -import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockTransferClient, DownloadFileManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus @@ -94,7 +94,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, - transferService: Option[BlockTransferService] = Option.empty, + blockTransferClientSync: Option[BlockTransferClientSync] = Option.empty, testConf: Option[SparkConf] = None): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) @@ -106,12 +106,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE None } val bmSecurityMgr = new SecurityManager(bmConf, encryptionKey) - val transfer = transferService + val transfer = blockTransferClientSync + .map(_.blockTransferClient.asInstanceOf[BlockTransferService]) .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)) val memManager = UnifiedMemoryManager(bmConf, numCores = 1) val serializerManager = new SerializerManager(serializer, bmConf) val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) + if (blockTransferClientSync.isDefined) { + blockManager.blockTransferClientSync = blockTransferClientSync.get + } memManager.setMemoryStore(blockManager.memoryStore) allStores += blockManager blockManager.initialize("app-id") @@ -1287,10 +1291,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { - val mockBlockTransferService = - new MockBlockTransferService(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) + val mockTransferClient = + new MockBlockTransferClientSync(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) val store = - makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("item").isEmpty) } @@ -1299,8 +1303,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val maxFailuresBeforeLocationRefresh = conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockBlockTransferService = - new MockBlockTransferService(maxFailuresBeforeLocationRefresh) + val mockTransferClient = new MockBlockTransferClientSync(maxFailuresBeforeLocationRefresh) // make sure we have more than maxFailuresBeforeLocationRefresh locations // so that we have a chance to do location refresh val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) @@ -1311,7 +1314,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManagerIds) val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, - transferService = Option(mockBlockTransferService)) + blockTransferClientSync = Option(mockTransferClient)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] assert(block.isDefined) @@ -1320,7 +1323,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-17484: block status is properly updated following an exception in put()") { - val mockBlockTransferService = new MockBlockTransferService(maxFailures = 10) { + val mockTransferService = new MockBlockTransferService { override def uploadBlock( hostname: String, port: Int, execId: String, @@ -1331,10 +1334,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE throw new InterruptedException("Intentional interrupt") } } + + val mockTransferClient = new MockBlockTransferClientSync(maxFailures = 10, mockTransferService) val store = - makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) val store2 = - makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + makeBlockManager(8000, "executor2", blockTransferClientSync = Option(mockTransferClient)) intercept[InterruptedException] { store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) } @@ -1354,6 +1359,36 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("item").isEmpty) } + test("test sorting of block locations") { + val localHost = "localhost" + val otherHost = "otherHost" + val store = makeBlockManager(8000, "executor1") + val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf) + val port = store.blockTransferService.port + val rack = Some("rack") + val blockManagerWithTopolgyInfo = BlockManagerId( + store.blockManagerId.executorId, + store.blockManagerId.host, + store.blockManagerId.port, + rack) + store.blockManagerId = blockManagerWithTopolgyInfo + val locations = Seq( + BlockManagerId("executor4", otherHost, externalShuffleServicePort, rack), + BlockManagerId("executor3", otherHost, port, rack), + BlockManagerId("executor6", otherHost, externalShuffleServicePort), + BlockManagerId("executor5", otherHost, port), + BlockManagerId("executor2", localHost, externalShuffleServicePort), + BlockManagerId("executor1", localHost, port)) + val sortedLocations = Seq( + BlockManagerId("executor1", localHost, port), + BlockManagerId("executor2", localHost, externalShuffleServicePort), + BlockManagerId("executor3", otherHost, port, rack), + BlockManagerId("executor4", otherHost, externalShuffleServicePort, rack), + BlockManagerId("executor5", otherHost, port), + BlockManagerId("executor6", otherHost, externalShuffleServicePort)) + assert(store.sortLocations(locations) === sortedLocations) + } + test("SPARK-20640: Shuffle registration timeout and maxAttempts conf are working") { val tryAgainMsg = "test_spark_20640_try_again" val timingoutExecutor = "timingoutExecutor" @@ -1436,7 +1471,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set(MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM, 1000L) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockBlockTransferService = new MockBlockTransferService(0) + val mockTransferClient = new MockBlockTransferClientSync(0) val blockLocations = Seq(BlockManagerId("id-0", "host-0", 1)) val blockStatus = BlockStatus(StorageLevel.DISK_ONLY, 0L, 2000L) @@ -1445,14 +1480,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations) val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, - transferService = Option(mockBlockTransferService)) + blockTransferClientSync = Option(mockTransferClient)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] assert(block.isDefined) - assert(mockBlockTransferService.numCalls === 1) + assert(mockTransferClient.numCalls === 1) // assert FileManager is not null if the block size is larger than threshold. - assert(mockBlockTransferService.tempFileManager === store.remoteBlockTempFileManager) + assert(mockTransferClient.tempFileManager === store.remoteBlockTempFileManager) } test("query locations of blockIds") { @@ -1468,9 +1503,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(locs(blockIds(0)) == expectedLocs) } - class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { - var numCalls = 0 - var tempFileManager: DownloadFileManager = null + class MockBlockTransferService extends BlockTransferService { override def init(blockDataManager: BlockDataManager): Unit = {} @@ -1501,6 +1534,15 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE import scala.concurrent.ExecutionContext.Implicits.global Future {} } + } + + class MockBlockTransferClientSync( + maxFailures: Int, + blockTransferClient: BlockTransferClient = new MockBlockTransferService()) + extends BlockTransferClientSync(blockTransferClient) { + + var numCalls = 0 + var tempFileManager: DownloadFileManager = null override def fetchBlockSync( host: String, From c6d81ec21cb7b0c78b0ae3219770fb6ed96fd677 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 30 Apr 2019 22:43:46 +0200 Subject: [PATCH 02/25] applying review comments 1.0 --- .../shuffle/ExternalShuffleBlockHandler.java | 2 +- ...tSync.scala => SyncBlockTransferClient.scala} | 8 ++++++-- .../org/apache/spark/storage/BlockManager.scala | 2 +- .../storage/BlockManagerMasterEndpoint.scala | 2 +- .../org/apache/spark/DistributedSuite.scala | 4 ++-- .../network/BlockTransferServiceSuite.scala | 2 +- .../apache/spark/storage/BlockManagerSuite.scala | 16 ++++++++-------- 7 files changed, 20 insertions(+), 16 deletions(-) rename core/src/main/scala/org/apache/spark/network/{BlockTransferClientSync.scala => SyncBlockTransferClient.scala} (93%) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 42382b9df107..b4a23a9caf46 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -225,7 +225,7 @@ private class ManagedBufferIterator implements Iterator { size = mapIdAndReduceIds.length; blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); - } else if(blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { + } else if (blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { final int[] rddAndSplitIds = rddAndSplitIds(blockIds); size = rddAndSplitIds.length; blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala b/core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala rename to core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala index 65225fb781bf..5ba7e9b265e1 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferClientSync.scala +++ b/core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala @@ -26,14 +26,18 @@ import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockTransferCli import org.apache.spark.storage.EncryptedManagedBuffer import org.apache.spark.util.ThreadUtils -private[spark] class BlockTransferClientSync(val blockTransferClient: BlockTransferClient) { +private[spark] class SyncBlockTransferClient(val blockTransferClient: BlockTransferClient) { /** * A special case of fetchBlocks, as it fetches only one block and is blocking. * * It is also only available after the BlockTransferClient is initialised. */ - def fetchBlockSync(host: String, port: Int, execId: String, blockId: String, + def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String, tempFileManager: DownloadFileManager): ManagedBuffer = { // A monitor for the thread to wait on. val result = Promise[ManagedBuffer]() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8baeb7ca1b08..b18713b54435 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -139,7 +139,7 @@ private[spark] class BlockManager( conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION) // Visible for testing - private[storage] var blockTransferClientSync = new BlockTransferClientSync(blockTransferService) + private[storage] var blockTransferClientSync = new SyncBlockTransferClient(blockTransferService) val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 0737926385e8..280c1208e4c1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -549,7 +549,7 @@ private[spark] class BlockManagerInfo( /** * Cached blocks held exclusively by this BlockManager. This does not include broadcast blocks - * and local disc persisted blocks when external shuffle service is enabled. + * and local disk persisted blocks when external shuffle service is enabled. */ private val _exclusiveCachedBlocks = new mutable.HashSet[BlockId] diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 4c96a0328eee..2658e4f76c8f 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests._ -import org.apache.spark.network.BlockTransferClientSync +import org.apache.spark.network.SyncBlockTransferClient import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -186,7 +186,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager val blockTransfer = blockManager.blockTransferService - val blockTransferClientSync = new BlockTransferClientSync(blockTransfer) + val blockTransferClientSync = new SyncBlockTransferClient(blockTransfer) val serializerManager = SparkEnv.get.serializerManager val locations = blockManager.master.getLocations(blockId) assert(locations.size === storageLevel.replication, diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala index b603034c525a..8ccf1adeace3 100644 --- a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala @@ -93,7 +93,7 @@ class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits { } } - val blockTransferClientSync = new BlockTransferClientSync(blockTransferService) + val blockTransferClientSync = new SyncBlockTransferClient(blockTransferService) val e = intercept[SparkException] { failAfter(10.seconds) { blockTransferClientSync.fetchBlockSync("localhost-unused", 0, "exec-id-unused", diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 44e34a390fda..c6576d8a42d3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager -import org.apache.spark.network.{BlockDataManager, BlockTransferClientSync, BlockTransferService, TransportContext} +import org.apache.spark.network.{BlockDataManager, BlockTransferService, SyncBlockTransferClient, TransportContext} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} @@ -94,7 +94,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, - blockTransferClientSync: Option[BlockTransferClientSync] = Option.empty, + blockTransferClientSync: Option[SyncBlockTransferClient] = Option.empty, testConf: Option[SparkConf] = None): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) @@ -1292,7 +1292,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { val mockTransferClient = - new MockBlockTransferClientSync(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) + new MockSyncBlockTransferClient(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) val store = makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -1303,7 +1303,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val maxFailuresBeforeLocationRefresh = conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockTransferClient = new MockBlockTransferClientSync(maxFailuresBeforeLocationRefresh) + val mockTransferClient = new MockSyncBlockTransferClient(maxFailuresBeforeLocationRefresh) // make sure we have more than maxFailuresBeforeLocationRefresh locations // so that we have a chance to do location refresh val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) @@ -1335,7 +1335,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } - val mockTransferClient = new MockBlockTransferClientSync(maxFailures = 10, mockTransferService) + val mockTransferClient = new MockSyncBlockTransferClient(maxFailures = 10, mockTransferService) val store = makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) val store2 = @@ -1471,7 +1471,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set(MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM, 1000L) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockTransferClient = new MockBlockTransferClientSync(0) + val mockTransferClient = new MockSyncBlockTransferClient(0) val blockLocations = Seq(BlockManagerId("id-0", "host-0", 1)) val blockStatus = BlockStatus(StorageLevel.DISK_ONLY, 0L, 2000L) @@ -1536,10 +1536,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } - class MockBlockTransferClientSync( + class MockSyncBlockTransferClient( maxFailures: Int, blockTransferClient: BlockTransferClient = new MockBlockTransferService()) - extends BlockTransferClientSync(blockTransferClient) { + extends SyncBlockTransferClient(blockTransferClient) { var numCalls = 0 var tempFileManager: DownloadFileManager = null From 528b05a9f5109fb6b07213a99ab6e1879e4883eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 30 Apr 2019 23:23:01 +0200 Subject: [PATCH 03/25] java checkstyle fix --- .../org/apache/spark/network/shuffle/BlockTransferClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java index 30f6e6f97035..8dce270f3df7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.network.shuffle; +package org.apache.spark.network.shuffle; public interface BlockTransferClient { From 98d2cbc203dcaf8e34bc4e532b92d3e2cc1748b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 30 Apr 2019 23:25:42 +0200 Subject: [PATCH 04/25] java checkstyle fix 2.0 --- .../org/apache/spark/network/shuffle/BlockTransferClient.java | 1 + 1 file changed, 1 insertion(+) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java index 8dce270f3df7..88e377c3999f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.network.shuffle; public interface BlockTransferClient { From d5a31494ea613e88dab6d687fbca4de1c296491e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 30 Apr 2019 23:53:00 +0200 Subject: [PATCH 05/25] java checkstyle fix 3.0 --- .../spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index c48cb6f92941..e643ca3a1ae1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -49,7 +49,7 @@ public class ExternalShuffleBlockHandlerSuite { OneForOneStreamManager streamManager; ExternalShuffleBlockResolver blockResolver; RpcHandler handler; - ManagedBuffer blockMarkers[] = { + ManagedBuffer[] blockMarkers = { new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) }; From 00d456a2f48587bbc26f3b26b843bf86fcb95855 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 1 May 2019 21:44:05 +0200 Subject: [PATCH 06/25] SyncBlockTransferClient --- .../network/shuffle/BlockTransferClient.java | 32 --------- .../spark/network/shuffle/ShuffleClient.java | 2 +- .../spark/network/BlockTransferService.scala | 46 ++++++++++++- .../network/SyncBlockTransferClient.scala | 69 ------------------- .../apache/spark/storage/BlockManager.scala | 5 +- .../org/apache/spark/DistributedSuite.scala | 10 +-- .../network/BlockTransferServiceSuite.scala | 5 +- .../spark/storage/BlockManagerSuite.scala | 52 ++++++-------- 8 files changed, 69 insertions(+), 152 deletions(-) delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java delete mode 100644 core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java deleted file mode 100644 index 88e377c3999f..000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockTransferClient.java +++ /dev/null @@ -1,32 +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.network.shuffle; - -public interface BlockTransferClient { - - /** - * Fetch a sequence of blocks from a remote node asynchronously. - */ - void fetchBlocks( - String host, - int port, - String execId, - String[] blockIds, - BlockFetchingListener listener, - DownloadFileManager downloadFileManager); -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index cea637d066fa..62b99c40f61f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -23,7 +23,7 @@ import com.codahale.metrics.MetricSet; /** Provides an interface for reading shuffle files, either from an Executor or external service. */ -public abstract class ShuffleClient implements BlockTransferClient, Closeable { +public abstract class ShuffleClient implements Closeable { /** * Initializes the ShuffleClient, specifying this Executor's appId. diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index c5f2f85512fd..51ced697c6af 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -18,15 +18,16 @@ package org.apache.spark.network import java.io.Closeable +import java.nio.ByteBuffer -import scala.concurrent.{Future} +import scala.concurrent.{Future, Promise} import scala.concurrent.duration.Duration import scala.reflect.ClassTag import org.apache.spark.internal.Logging -import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, EncryptedManagedBuffer, StorageLevel} import org.apache.spark.util.ThreadUtils private[spark] @@ -81,6 +82,45 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo level: StorageLevel, classTag: ClassTag[_]): Future[Unit] + /** + * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. + * + * It is also only available after [[init]] is invoked. + */ + def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String, + tempFileManager: DownloadFileManager): ManagedBuffer = { + // A monitor for the thread to wait on. + val result = Promise[ManagedBuffer]() + fetchBlocks(host, port, execId, Array(blockId), + new BlockFetchingListener { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + result.failure(exception) + } + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + data match { + case f: FileSegmentManagedBuffer => + result.success(f) + case e: EncryptedManagedBuffer => + result.success(e) + case _ => + try { + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result.success(new NioManagedBuffer(ret)) + } catch { + case e: Throwable => result.failure(e) + } + } + } + }, tempFileManager) + ThreadUtils.awaitResult(result.future, Duration.Inf) + } + /** * Upload a single block to a remote node, available only after [[init]] is invoked. * diff --git a/core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala b/core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala deleted file mode 100644 index 5ba7e9b265e1..000000000000 --- a/core/src/main/scala/org/apache/spark/network/SyncBlockTransferClient.scala +++ /dev/null @@ -1,69 +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.network - -import java.nio.ByteBuffer - -import scala.concurrent.Promise -import scala.concurrent.duration.Duration - -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockTransferClient, DownloadFileManager} -import org.apache.spark.storage.EncryptedManagedBuffer -import org.apache.spark.util.ThreadUtils - -private[spark] class SyncBlockTransferClient(val blockTransferClient: BlockTransferClient) { - - /** - * A special case of fetchBlocks, as it fetches only one block and is blocking. - * - * It is also only available after the BlockTransferClient is initialised. - */ - def fetchBlockSync( - host: String, - port: Int, - execId: String, - blockId: String, - tempFileManager: DownloadFileManager): ManagedBuffer = { - // A monitor for the thread to wait on. - val result = Promise[ManagedBuffer]() - blockTransferClient.fetchBlocks(host, port, execId, Array(blockId), - new BlockFetchingListener { - override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - result.failure(exception) - } - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - data match { - case f: FileSegmentManagedBuffer => - result.success(f) - case e: EncryptedManagedBuffer => - result.success(e) - case _ => - try { - val ret = ByteBuffer.allocate(data.size.toInt) - ret.put(data.nioByteBuffer()) - ret.flip() - result.success(new NioManagedBuffer(ret)) - } catch { - case e: Throwable => result.failure(e) - } - } - } - }, tempFileManager) - ThreadUtils.awaitResult(result.future, Duration.Inf) - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b18713b54435..5066865fd248 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -138,9 +138,6 @@ private[spark] class BlockManager( private val remoteReadNioBufferConversion = conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION) - // Visible for testing - private[storage] var blockTransferClientSync = new SyncBlockTransferClient(blockTransferService) - val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. val deleteFilesOnStop = @@ -901,7 +898,7 @@ private[spark] class BlockManager( val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { - blockTransferClientSync.fetchBlockSync(loc.host, loc.port, loc.executorId, blockId.toString, + blockTransferService.fetchBlockSync(loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager) } catch { case NonFatal(e) => diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 2658e4f76c8f..aad20545bafb 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests._ -import org.apache.spark.network.SyncBlockTransferClient import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -186,18 +185,13 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager val blockTransfer = blockManager.blockTransferService - val blockTransferClientSync = new SyncBlockTransferClient(blockTransfer) val serializerManager = SparkEnv.get.serializerManager val locations = blockManager.master.getLocations(blockId) assert(locations.size === storageLevel.replication, s"; got ${locations.size} replicas instead of ${storageLevel.replication}") locations.foreach { cmId => - val bytes = blockTransferClientSync.fetchBlockSync( - cmId.host, - cmId.port, - cmId.executorId, - blockId.toString, - null) + val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, + blockId.toString, null) val deserialized = serializerManager.dataDeserializeStream(blockId, new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala index 8ccf1adeace3..d7e4b9166fa0 100644 --- a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala @@ -93,11 +93,10 @@ class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits { } } - val blockTransferClientSync = new SyncBlockTransferClient(blockTransferService) val e = intercept[SparkException] { failAfter(10.seconds) { - blockTransferClientSync.fetchBlockSync("localhost-unused", 0, "exec-id-unused", - "block-id-unused", null) + blockTransferService.fetchBlockSync( + "localhost-unused", 0, "exec-id-unused", "block-id-unused", null) } } assert(e.getCause.isInstanceOf[IllegalArgumentException]) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c6576d8a42d3..771ba32207c4 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -39,12 +39,12 @@ import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager -import org.apache.spark.network.{BlockDataManager, BlockTransferService, SyncBlockTransferClient, TransportContext} +import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} -import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockTransferClient, DownloadFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus @@ -94,7 +94,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, - blockTransferClientSync: Option[SyncBlockTransferClient] = Option.empty, + transferService: Option[BlockTransferService] = Option.empty, testConf: Option[SparkConf] = None): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) bmConf.set(TEST_MEMORY, maxMem) @@ -106,16 +106,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE None } val bmSecurityMgr = new SecurityManager(bmConf, encryptionKey) - val transfer = blockTransferClientSync - .map(_.blockTransferClient.asInstanceOf[BlockTransferService]) + val transfer = transferService .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)) val memManager = UnifiedMemoryManager(bmConf, numCores = 1) val serializerManager = new SerializerManager(serializer, bmConf) val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) - if (blockTransferClientSync.isDefined) { - blockManager.blockTransferClientSync = blockTransferClientSync.get - } memManager.setMemoryStore(blockManager.memoryStore) allStores += blockManager blockManager.initialize("app-id") @@ -1291,10 +1287,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { - val mockTransferClient = - new MockSyncBlockTransferClient(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) + val mockBlockTransferService = + new MockBlockTransferService(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) val store = - makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("item").isEmpty) } @@ -1303,7 +1299,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val maxFailuresBeforeLocationRefresh = conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockTransferClient = new MockSyncBlockTransferClient(maxFailuresBeforeLocationRefresh) + val mockBlockTransferService = + new MockBlockTransferService(maxFailuresBeforeLocationRefresh) // make sure we have more than maxFailuresBeforeLocationRefresh locations // so that we have a chance to do location refresh val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) @@ -1314,7 +1311,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE blockManagerIds) val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, - blockTransferClientSync = Option(mockTransferClient)) + transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] assert(block.isDefined) @@ -1323,7 +1320,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-17484: block status is properly updated following an exception in put()") { - val mockTransferService = new MockBlockTransferService { + val mockBlockTransferService = new MockBlockTransferService(maxFailures = 10) { override def uploadBlock( hostname: String, port: Int, execId: String, @@ -1334,12 +1331,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE throw new InterruptedException("Intentional interrupt") } } - - val mockTransferClient = new MockSyncBlockTransferClient(maxFailures = 10, mockTransferService) val store = - makeBlockManager(8000, "executor1", blockTransferClientSync = Option(mockTransferClient)) + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) val store2 = - makeBlockManager(8000, "executor2", blockTransferClientSync = Option(mockTransferClient)) + makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) intercept[InterruptedException] { store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) } @@ -1471,7 +1466,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set(MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM, 1000L) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) - val mockTransferClient = new MockSyncBlockTransferClient(0) + val mockBlockTransferService = new MockBlockTransferService(0) val blockLocations = Seq(BlockManagerId("id-0", "host-0", 1)) val blockStatus = BlockStatus(StorageLevel.DISK_ONLY, 0L, 2000L) @@ -1480,14 +1475,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations) val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, - blockTransferClientSync = Option(mockTransferClient)) + transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] assert(block.isDefined) - assert(mockTransferClient.numCalls === 1) + assert(mockBlockTransferService.numCalls === 1) // assert FileManager is not null if the block size is larger than threshold. - assert(mockTransferClient.tempFileManager === store.remoteBlockTempFileManager) + assert(mockBlockTransferService.tempFileManager === store.remoteBlockTempFileManager) } test("query locations of blockIds") { @@ -1503,7 +1498,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(locs(blockIds(0)) == expectedLocs) } - class MockBlockTransferService extends BlockTransferService { + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { + var numCalls = 0 + var tempFileManager: DownloadFileManager = null override def init(blockDataManager: BlockDataManager): Unit = {} @@ -1534,15 +1531,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE import scala.concurrent.ExecutionContext.Implicits.global Future {} } - } - - class MockSyncBlockTransferClient( - maxFailures: Int, - blockTransferClient: BlockTransferClient = new MockBlockTransferService()) - extends SyncBlockTransferClient(blockTransferClient) { - - var numCalls = 0 - var tempFileManager: DownloadFileManager = null override def fetchBlockSync( host: String, From 37ca716e3ed63281e739199d133cfa7c8e35a65b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 1 May 2019 21:47:14 +0200 Subject: [PATCH 07/25] Handling of deleted files --- .../server/ChunkFetchRequestHandler.java | 3 ++ .../server/OneForOneStreamManager.java | 5 +- .../ChunkFetchRequestHandlerSuite.java | 42 ++++++++++------- .../network/TransportRequestHandlerSuite.java | 47 +++++++++++-------- .../server/OneForOneStreamManagerSuite.java | 34 ++++++++++++-- .../shuffle/ExternalShuffleBlockResolver.java | 7 ++- .../ExternalShuffleBlockHandlerSuite.java | 23 +++++++-- .../apache/spark/deploy/worker/Worker.scala | 12 ++++- .../apache/spark/storage/BlockManager.scala | 8 +++- .../spark/storage/BlockManagerSuite.scala | 26 +++++++++- 10 files changed, 159 insertions(+), 48 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java index 43c3d23b6304..94412c4db559 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java @@ -91,6 +91,9 @@ protected void channelRead0( try { streamManager.checkAuthorization(client, msg.streamChunkId.streamId); buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex); + if (buf == null) { + throw new IllegalStateException("Chunk was not found"); + } } catch (Exception e) { logger.error(String.format("Error opening block %s for request from %s", msg.streamChunkId, getRemoteAddress(channel)), e); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 6fafcc131fa2..67f64d796203 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -125,7 +125,10 @@ public void connectionTerminated(Channel channel) { // Release all remaining buffers. while (state.buffers.hasNext()) { - state.buffers.next().release(); + ManagedBuffer buffer = state.buffers.next(); + if (buffer != null) { + buffer.release(); + } } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java index 6c9239606bb8..7e30ed4048ca 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java @@ -23,6 +23,7 @@ import io.netty.channel.Channel; import org.apache.spark.network.server.ChunkFetchRequestHandler; +import org.junit.Assert; import org.junit.Test; import static org.mockito.Mockito.*; @@ -45,9 +46,8 @@ public void handleChunkFetchRequest() throws Exception { Channel channel = mock(Channel.class); ChannelHandlerContext context = mock(ChannelHandlerContext.class); when(context.channel()) - .thenAnswer(invocationOnMock0 -> { - return channel; - }); + .thenAnswer(invocationOnMock0 -> channel); + List> responseAndPromisePairs = new ArrayList<>(); when(channel.writeAndFlush(any())) @@ -62,6 +62,7 @@ public void handleChunkFetchRequest() throws Exception { List managedBuffers = new ArrayList<>(); managedBuffers.add(new TestManagedBuffer(10)); managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(null); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); @@ -71,31 +72,40 @@ public void handleChunkFetchRequest() throws Exception { RequestMessage request0 = new ChunkFetchRequest(new StreamChunkId(streamId, 0)); requestHandler.channelRead(context, request0); - assert responseAndPromisePairs.size() == 1; - assert responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body() == - managedBuffers.get(0); + Assert.assertEquals(1, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(0), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body()); RequestMessage request1 = new ChunkFetchRequest(new StreamChunkId(streamId, 1)); requestHandler.channelRead(context, request1); - assert responseAndPromisePairs.size() == 2; - assert responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body() == - managedBuffers.get(1); + Assert.assertEquals(2, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(1), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body()); // Finish flushing the response for request0. responseAndPromisePairs.get(0).getRight().finish(true); RequestMessage request2 = new ChunkFetchRequest(new StreamChunkId(streamId, 2)); requestHandler.channelRead(context, request2); - assert responseAndPromisePairs.size() == 3; - assert responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(2).getLeft())).body() == - managedBuffers.get(2); + Assert.assertEquals(3, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchFailure); + ChunkFetchFailure chunkFetchFailure = + ((ChunkFetchFailure) (responseAndPromisePairs.get(2).getLeft())); + Assert.assertEquals("java.lang.IllegalStateException: Chunk was not found", + chunkFetchFailure.errorString.split("\\r?\\n")[0]); RequestMessage request3 = new ChunkFetchRequest(new StreamChunkId(streamId, 3)); requestHandler.channelRead(context, request3); + Assert.assertEquals(4, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(3), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(3).getLeft())).body()); + + RequestMessage request4 = new ChunkFetchRequest(new StreamChunkId(streamId, 4)); + requestHandler.channelRead(context, request4); verify(channel, times(1)).close(); - assert responseAndPromisePairs.size() == 3; + Assert.assertEquals(4, responseAndPromisePairs.size()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java index a87f6c11a2bf..a43a65904868 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java @@ -21,6 +21,7 @@ import java.util.List; import io.netty.channel.Channel; +import org.junit.Assert; import org.junit.Test; import static org.mockito.Mockito.*; @@ -38,7 +39,7 @@ public class TransportRequestHandlerSuite { @Test - public void handleStreamRequest() throws Exception { + public void handleStreamRequest() { RpcHandler rpcHandler = new NoOpRpcHandler(); OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager()); Channel channel = mock(Channel.class); @@ -56,11 +57,12 @@ public void handleStreamRequest() throws Exception { List managedBuffers = new ArrayList<>(); managedBuffers.add(new TestManagedBuffer(10)); managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(null); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); - assert streamManager.numStreamStates() == 1; + Assert.assertEquals(1, streamManager.numStreamStates()); TransportClient reverseClient = mock(TransportClient.class); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, @@ -68,36 +70,43 @@ public void handleStreamRequest() throws Exception { RequestMessage request0 = new StreamRequest(String.format("%d_%d", streamId, 0)); requestHandler.handle(request0); - assert responseAndPromisePairs.size() == 1; - assert responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body() == - managedBuffers.get(0); + Assert.assertEquals(1, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(0), + ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body()); RequestMessage request1 = new StreamRequest(String.format("%d_%d", streamId, 1)); requestHandler.handle(request1); - assert responseAndPromisePairs.size() == 2; - assert responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body() == - managedBuffers.get(1); + Assert.assertEquals(2, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(1), + ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body()); // Finish flushing the response for request0. responseAndPromisePairs.get(0).getRight().finish(true); - RequestMessage request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); + StreamRequest request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); requestHandler.handle(request2); - assert responseAndPromisePairs.size() == 3; - assert responseAndPromisePairs.get(2).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(2).getLeft())).body() == - managedBuffers.get(2); + Assert.assertEquals(3, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof StreamFailure); + Assert.assertEquals(String.format("Stream '%s' was not found.", request2.streamId), + ((StreamFailure) (responseAndPromisePairs.get(2).getLeft())).error); - // Request3 will trigger the close of channel, because the number of max chunks being - // transferred is 2; RequestMessage request3 = new StreamRequest(String.format("%d_%d", streamId, 3)); requestHandler.handle(request3); + Assert.assertEquals(4, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(3), + ((StreamResponse) (responseAndPromisePairs.get(3).getLeft())).body()); + + // Request4 will trigger the close of channel, because the number of max chunks being + // transferred is 2; + RequestMessage request4 = new StreamRequest(String.format("%d_%d", streamId, 4)); + requestHandler.handle(request4); verify(channel, times(1)).close(); - assert responseAndPromisePairs.size() == 3; + Assert.assertEquals(4, responseAndPromisePairs.size()); streamManager.connectionTerminated(channel); - assert streamManager.numStreamStates() == 0; + Assert.assertEquals(0, streamManager.numStreamStates()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index 4248762c3238..a0ddea462210 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -21,6 +21,7 @@ import java.util.List; import io.netty.channel.Channel; +import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -30,22 +31,47 @@ public class OneForOneStreamManagerSuite { @Test - public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { + public void testMissingChunk() { OneForOneStreamManager manager = new OneForOneStreamManager(); List buffers = new ArrayList<>(); TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + TestManagedBuffer buffer3 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(null); buffers.add(buffer2); + buffers.add(null); + buffers.add(buffer3); Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); - manager.registerStream("appId", buffers.iterator(), dummyChannel); - assert manager.numStreamStates() == 1; + long streamId = manager.registerStream("appId", buffers.iterator(), dummyChannel); + Assert.assertEquals(1, manager.numStreamStates()); + Assert.assertNotNull(manager.getChunk(streamId, 0)); + Assert.assertNull(manager.getChunk(streamId, 1)); + Assert.assertNotNull(manager.getChunk(streamId, 2)); + manager.connectionTerminated(dummyChannel); + Mockito.verify(buffer1, Mockito.never()).release(); + Mockito.verify(buffer2, Mockito.never()).release(); + Mockito.verify(buffer3, Mockito.times(1)).release(); + } + + @Test + public void managedBuffersAreFeedWhenConnectionIsClosed() { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(buffer2); + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerStream("appId", buffers.iterator(), dummyChannel); + Assert.assertEquals(1, manager.numStreamStates()); manager.connectionTerminated(dummyChannel); Mockito.verify(buffer1, Mockito.times(1)).release(); Mockito.verify(buffer2, Mockito.times(1)).release(); - assert manager.numStreamStates() == 0; + Assert.assertEquals(0, manager.numStreamStates()); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 70cb115278ca..f44407194a2b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -314,7 +314,12 @@ public ManagedBuffer getDiskPersistedRddBlockData( ExecutorShuffleInfo executor, int rddId, int splitIndex) { File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, "rdd_" + rddId + "_" + splitIndex); - return new FileSegmentManagedBuffer(conf, file, 0, file.length()); + long length = file.length(); + ManagedBuffer res = null; + if (file.exists()) { + res = new FileSegmentManagedBuffer(conf, file, 0, length); + } + return res; } /** diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index e643ca3a1ae1..5d01b96f1ec4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -85,7 +85,7 @@ public void testOpenShuffleBlocks() { when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(blockMarkers[0]); when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); - checkOpenBlocksReceive(new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }); + checkOpenBlocksReceive(new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); @@ -97,14 +97,31 @@ public void testOpenDiskPersistedRDDBlocks() { when(blockResolver.getBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); when(blockResolver.getBlockData("app0", "exec1", 0, 1)).thenReturn(blockMarkers[1]); - checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }); + checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 1); verifyOpenBlockLatencyMetrics(); } - private void checkOpenBlocksReceive(String[] blockIds) { + @Test + public void testOpenDiskPersistedRDDBlocksWithMissingBlock() { + ManagedBuffer[] blockMarkersWithMissingBlock = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), + null + }; + when(blockResolver.getBlockData("app0", "exec1", 0, 0)) + .thenReturn(blockMarkersWithMissingBlock[0]); + when(blockResolver.getBlockData("app0", "exec1", 0, 1)) + .thenReturn(null); + + checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }, blockMarkersWithMissingBlock); + + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 1); + } + + private void checkOpenBlocksReceive(String[] blockIds, ManagedBuffer[] blockMarkers) { when(client.getClientId()).thenReturn("app0"); RpcResponseCallback callback = mock(RpcResponseCallback.class); diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f8ec5b6b190c..35ea887eb487 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -104,6 +104,15 @@ private[deploy] class Worker( private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = conf.get(config.STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT) + val EXTERNAL_SHUFFLE_SERVICE_ENABLED = conf.get(config.SHUFFLE_SERVICE_ENABLED) + + if (CLEANUP_NON_SHUFFLE_FILES_ENABLED && EXTERNAL_SHUFFLE_SERVICE_ENABLED) { + logWarning("Both 'spark.storage.cleanupFilesAfterExecutorExit' and " + + "'spark.shuffle.service.enabled' are switched on. But with SPARK-25888 the external " + + "shuffle service able to serve disk persisted RDD blocks. So to keep benefiting from " + + "SPARK-25888 the cleanup only will be triggered after the application is stopped.") + } + private var master: Option[RpcEndpointRef] = None /** @@ -750,7 +759,8 @@ private[deploy] class Worker( trimFinishedExecutorsIfNecessary() coresUsed -= executor.cores memoryUsed -= executor.memory - if (CLEANUP_NON_SHUFFLE_FILES_ENABLED) { + + if (CLEANUP_NON_SHUFFLE_FILES_ENABLED && !EXTERNAL_SHUFFLE_SERVICE_ENABLED) { shuffleService.executorRemoved(executorStateChanged.execId.toString, appId) } case None => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5066865fd248..8502f44fa2ce 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -898,8 +898,12 @@ private[spark] class BlockManager( val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { - blockTransferService.fetchBlockSync(loc.host, loc.port, loc.executorId, blockId.toString, - tempFileManager) + val buf = blockTransferService.fetchBlockSync(loc.host, loc.port, loc.executorId, + blockId.toString, tempFileManager) + if (blockSize > 0 && buf.size() == 0) { + throw new IllegalStateException("Empty buffer received for non empty block") + } + buf } catch { case NonFatal(e) => runningFailureCount += 1 diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 771ba32207c4..731ad1a69a81 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.storage +import java.io.File import java.nio.ByteBuffer import scala.collection.JavaConverters._ @@ -40,7 +41,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} @@ -1354,6 +1355,29 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("item").isEmpty) } + test("SPARK-25888: serving of removed file not detected by shuffle service") { + // although the existence of the file is checked before serving it but a delete can happen + // somewhere after that check + val store = makeBlockManager(8000, "executor1") + val emptyBlockFetcher = new MockBlockTransferService(0) { + override def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String, + tempFileManager: DownloadFileManager): ManagedBuffer = { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 1) + // empty ManagedBuffer + new FileSegmentManagedBuffer(transConf, new File("missing.file"), 0, 0) + } + } + val store2 = + makeBlockManager(8000, "executor2", this.master, Some(emptyBlockFetcher)) + store.putSingle("item", "value", StorageLevel.DISK_ONLY, tellMaster = true) + assert(master.getLocations("item").nonEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + } + test("test sorting of block locations") { val localHost = "localhost" val otherHost = "otherHost" From fd0b107e8b14636043c5d81f26d6140f6fa1d046 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Fri, 3 May 2019 03:58:13 +0200 Subject: [PATCH 08/25] mostly indentation fixes --- .../spark/ExternalShuffleServiceSuite.scala | 10 +- .../spark/storage/BlockManagerInfoSuite.scala | 176 +++++++++--------- 2 files changed, 85 insertions(+), 101 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 0908add2f008..b4cb7fcb66d9 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -101,16 +101,8 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi sc.env.blockManager.externalShuffleServiceEnabled should equal(true) sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) - // In a slow machine, one slave may register hundreds of milliseconds ahead of the other one. - // If we don't wait for all slaves, it's possible that only one executor runs all jobs. Then - // all shuffle blocks will be in this executor, ShuffleBlockFetcherIterator will directly fetch - // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. - // In this case, we won't receive FetchFailed. And it will make this test fail. - // Therefore, we should wait until all slaves are up - TestUtils.waitUntilExecutorsUp(sc, 1, 60000) - val rdd = sc.parallelize(0 until 100, 2) - .map(i => (i, 1)) + .map { i => (i, 1) } .persist(StorageLevel.DISK_ONLY) rdd.count() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index d0329fa29fde..7cb30c0b24b8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -27,141 +27,133 @@ class BlockManagerInfoSuite extends SparkFunSuite { def testWithShuffleServiceOnOff(testName: String) (f: (Boolean, BlockManagerInfo) => Unit): Unit = { - Seq(true, false).foreach { shuffleServiceEnabled => - val blockManagerInfo = new BlockManagerInfo( + Seq(true, false).foreach { svcEnabled => + val bmInfo = new BlockManagerInfo( BlockManagerId("executor0", "host", 1234, None), timeMs = 300, maxOnHeapMem = 10000, maxOffHeapMem = 20000, slaveEndpoint = null, - if (shuffleServiceEnabled) Some(new JHashMap[BlockId, BlockStatus]) else None) - test(s"$testName externalShuffleServiceEnabled=$shuffleServiceEnabled") { - f(shuffleServiceEnabled, blockManagerInfo) + if (svcEnabled) Some(new JHashMap[BlockId, BlockStatus]) else None) + test(s"$testName externalShuffleServiceEnabled=$svcEnabled") { + f(svcEnabled, bmInfo) } } } - testWithShuffleServiceOnOff("add broadcast block") { (_, blockManagerInfo) => + testWithShuffleServiceOnOff("broadcast block") { (_, bmInfo) => val broadcastId: BlockId = BroadcastBlockId(0, "field1") - blockManagerInfo.updateBlockInfo( + bmInfo.updateBlockInfo( broadcastId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 100) - assert(blockManagerInfo.blocks.asScala - === Map(broadcastId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 100))) - assert(blockManagerInfo.exclusiveCachedBlocks === Set()) - assert(blockManagerInfo.remainingMem == 29800) + assert(bmInfo.blocks.asScala === + Map(broadcastId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 100))) + assert(bmInfo.exclusiveCachedBlocks.isEmpty) + assert(bmInfo.remainingMem == 29800) } - testWithShuffleServiceOnOff("add RDD block with MEMORY_ONLY") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("RDD block with MEMORY_ONLY") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, diskSize = 0) - assert(blockManagerInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) - assert(blockManagerInfo.exclusiveCachedBlocks === Set(rddId)) - assert(blockManagerInfo.remainingMem == 29800) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, diskSize = 0) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(bmInfo.exclusiveCachedBlocks === Set(rddId)) + assert(bmInfo.remainingMem == 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } } - testWithShuffleServiceOnOff("add RDD block with MEMORY_AND_DISK") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo( + bmInfo.updateBlockInfo( rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) - assert(blockManagerInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) - val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) - assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(blockManagerInfo.remainingMem == 29800) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala - === Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(bmInfo.remainingMem == 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) } } - testWithShuffleServiceOnOff("add RDD block with DISK_ONLY") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("RDD block with DISK_ONLY") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(blockManagerInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) - val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) - assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) } } - testWithShuffleServiceOnOff("update RDD block from MEMORY_ONLY to DISK_ONLY") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("update from MEMORY_ONLY to DISK_ONLY") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) - assert(blockManagerInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) - assert(blockManagerInfo.exclusiveCachedBlocks === Set(rddId)) - assert(blockManagerInfo.remainingMem == 29800) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala == Map()) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(bmInfo.exclusiveCachedBlocks === Set(rddId)) + assert(bmInfo.remainingMem == 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala == Map()) } - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(blockManagerInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) - val exclusiveCachedBlocksForNoMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) - assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForNoMemoryOnly) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForNoMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForNoMemoryOnly) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) } } - testWithShuffleServiceOnOff("using invalid StorageLevel") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("using invalid StorageLevel") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(blockManagerInfo.blocks.asScala + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) - val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) - assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) } - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.NONE, memSize = 0, diskSize = 200) - assert(blockManagerInfo.blocks.asScala === Map()) - assert(blockManagerInfo.exclusiveCachedBlocks === Set()) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + bmInfo.updateBlockInfo(rddId, StorageLevel.NONE, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.isEmpty) + assert(bmInfo.exclusiveCachedBlocks.isEmpty) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) } } - testWithShuffleServiceOnOff("remove block") { - (shuffleServiceEnabled, blockManagerInfo) => + testWithShuffleServiceOnOff("remove block") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) - blockManagerInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(blockManagerInfo.blocks.asScala + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) - val exclusiveCachedBlocksForOneMemoryOnly = if (shuffleServiceEnabled) Set() else Set(rddId) - assert(blockManagerInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) } - blockManagerInfo.removeBlock(rddId) - assert(blockManagerInfo.blocks.asScala === Map()) - assert(blockManagerInfo.exclusiveCachedBlocks === Set()) - assert(blockManagerInfo.remainingMem == 30000) - if (shuffleServiceEnabled) { - assert(blockManagerInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + bmInfo.removeBlock(rddId) + assert(bmInfo.blocks.asScala.isEmpty) + assert(bmInfo.exclusiveCachedBlocks.isEmpty) + assert(bmInfo.remainingMem == 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } } } From e7e053972b0f6afb2029c53fed4ea2d0dd2b1d92 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Fri, 3 May 2019 16:50:22 +0200 Subject: [PATCH 09/25] cleanup of non shuffle service served files --- .../shuffle/ExternalShuffleBlockResolver.java | 31 ++++--- ...nupNonShuffleServiceServedFilesSuite.java} | 89 ++++++++----------- .../shuffle/TestShuffleDataContext.java | 20 ++++- .../apache/spark/deploy/worker/Worker.scala | 15 +--- .../spark/internal/config/package.scala | 2 +- .../storage/BlockManagerMasterEndpoint.scala | 2 +- 6 files changed, 75 insertions(+), 84 deletions(-) rename common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/{NonShuffleFilesCleanupSuite.java => CleanupNonShuffleServiceServedFilesSuite.java} (65%) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index f44407194a2b..21827b17e24b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -229,22 +229,23 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } /** - * Removes all the non-shuffle files in any local directories associated with the finished - * executor. + * Removes all the files which cannot be served by the external shuffle service (non-shuffle and + * non-RDD files) in any local directories associated with the finished executor. */ public void executorRemoved(String executorId, String appId) { - logger.info("Clean up non-shuffle files associated with the finished executor {}", executorId); + logger.info("Clean up non-shuffle and non-RDD files associated with the finished executor {}", + executorId); AppExecId fullId = new AppExecId(appId, executorId); final ExecutorShuffleInfo executor = executors.get(fullId); if (executor == null) { // Executor not registered, skip clean up of the local directories. logger.info("Executor is not registered (appId={}, execId={})", appId, executorId); } else { - logger.info("Cleaning up non-shuffle files in executor {}'s {} local dirs", fullId, - executor.localDirs.length); + logger.info("Cleaning up non-shuffle and non-RDD files in executor {}'s {} local dirs", + fullId, executor.localDirs.length); // Execute the actual deletion in a different thread, as it may take some time. - directoryCleaner.execute(() -> deleteNonShuffleFiles(executor.localDirs)); + directoryCleaner.execute(() -> deleteNonShuffleNonRddFiles(executor.localDirs)); } } @@ -264,24 +265,22 @@ private void deleteExecutorDirs(String[] dirs) { } /** - * Synchronously deletes non-shuffle files in each directory recursively. + * Synchronously deletes non-shuffle and non-RDD files in each directory recursively. * Should be executed in its own thread, as this may take a long time. */ - private void deleteNonShuffleFiles(String[] dirs) { - FilenameFilter filter = new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - // Don't delete shuffle data or shuffle index files. - return !name.endsWith(".index") && !name.endsWith(".data"); - } + private void deleteNonShuffleNonRddFiles(String[] dirs) { + FilenameFilter filter = (dir, name) -> { + // Don't delete shuffle data, shuffle index files or cached RDD files. + return !name.endsWith(".index") && !name.endsWith(".data") && !name.startsWith("rdd_"); }; for (String localDir : dirs) { try { JavaUtils.deleteRecursively(new File(localDir), filter); - logger.debug("Successfully cleaned up non-shuffle files in directory: {}", localDir); + logger.debug("Successfully cleaned up non-shuffle and non-RDD files in directory: {}", + localDir); } catch (Exception e) { - logger.error("Failed to delete non-shuffle files in directory: " + localDir, e); + logger.error("Failed to delete non-shuffle and non-RDD files in directory: " + localDir, e); } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java similarity index 65% rename from common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index d22f3ace4103..7aab494af395 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -32,7 +32,7 @@ import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; -public class NonShuffleFilesCleanupSuite { +public class CleanupNonShuffleServiceServedFilesSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); @@ -40,17 +40,17 @@ public class NonShuffleFilesCleanupSuite { private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; @Test - public void cleanupOnRemovedExecutorWithShuffleFiles() throws IOException { + public void cleanupOnRemovedExecutorWithFilesToKeep() throws IOException { cleanupOnRemovedExecutor(true); } @Test - public void cleanupOnRemovedExecutorWithoutShuffleFiles() throws IOException { + public void cleanupOnRemovedExecutorWithoutFilesToKeep() throws IOException { cleanupOnRemovedExecutor(false); } - private void cleanupOnRemovedExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); + private void cleanupOnRemovedExecutor(boolean withFilesToKeep) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); @@ -61,17 +61,17 @@ private void cleanupOnRemovedExecutor(boolean withShuffleFiles) throws IOExcepti } @Test - public void cleanupUsesExecutorWithShuffleFiles() throws IOException { + public void cleanupUsesExecutorWithFilesToKeep() throws IOException { cleanupUsesExecutor(true); } @Test - public void cleanupUsesExecutorWithoutShuffleFiles() throws IOException { + public void cleanupUsesExecutorWithoutFilesToKeep() throws IOException { cleanupUsesExecutor(false); } - private void cleanupUsesExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); + private void cleanupUsesExecutor(boolean withFilesToKeep) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); AtomicBoolean cleanupCalled = new AtomicBoolean(false); @@ -89,18 +89,18 @@ private void cleanupUsesExecutor(boolean withShuffleFiles) throws IOException { } @Test - public void cleanupOnlyRemovedExecutorWithShuffleFiles() throws IOException { + public void cleanupOnlyRemovedExecutorWithFilesToKeep() throws IOException { cleanupOnlyRemovedExecutor(true); } @Test - public void cleanupOnlyRemovedExecutorWithoutShuffleFiles() throws IOException { + public void cleanupOnlyRemovedExecutorWithoutFilesToKeep() throws IOException { cleanupOnlyRemovedExecutor(false); } - private void cleanupOnlyRemovedExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext0 = initDataContext(withShuffleFiles); - TestShuffleDataContext dataContext1 = initDataContext(withShuffleFiles); + private void cleanupOnlyRemovedExecutor(boolean withFilesToKeep) throws IOException { + TestShuffleDataContext dataContext0 = initDataContext(withFilesToKeep); + TestShuffleDataContext dataContext1 = initDataContext(withFilesToKeep); ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); @@ -127,17 +127,17 @@ private void cleanupOnlyRemovedExecutor(boolean withShuffleFiles) throws IOExcep } @Test - public void cleanupOnlyRegisteredExecutorWithShuffleFiles() throws IOException { + public void cleanupOnlyRegisteredExecutorWithFilesToKeep() throws IOException { cleanupOnlyRegisteredExecutor(true); } @Test - public void cleanupOnlyRegisteredExecutorWithoutShuffleFiles() throws IOException { + public void cleanupOnlyRegisteredExecutorWithoutFilesToKeep() throws IOException { cleanupOnlyRegisteredExecutor(false); } - private void cleanupOnlyRegisteredExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); + private void cleanupOnlyRegisteredExecutor(boolean withFilesToKeep) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); @@ -156,18 +156,15 @@ private static void assertStillThere(TestShuffleDataContext dataContext) { } } - private static FilenameFilter filter = new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - // Don't delete shuffle data or shuffle index files. - return !name.endsWith(".index") && !name.endsWith(".data"); - } + private static FilenameFilter filter = (dir, name) -> { + // Don't delete shuffle data or shuffle index files. + return !name.endsWith(".index") && !name.endsWith(".data") && !name.startsWith("rdd_"); }; - private static boolean assertOnlyShuffleDataInDir(File[] dirs) { + private static boolean assertOnlyFilesToKeepInDir(File[] dirs) { for (File dir : dirs) { assertTrue(dir.getName() + " wasn't cleaned up", !dir.exists() || - dir.listFiles(filter).length == 0 || assertOnlyShuffleDataInDir(dir.listFiles())); + dir.listFiles(filter).length == 0 || assertOnlyFilesToKeepInDir(dir.listFiles())); } return true; } @@ -175,47 +172,33 @@ private static boolean assertOnlyShuffleDataInDir(File[] dirs) { private static void assertCleanedUp(TestShuffleDataContext dataContext) { for (String localDir : dataContext.localDirs) { File[] dirs = new File[] {new File(localDir)}; - assertOnlyShuffleDataInDir(dirs); + assertOnlyFilesToKeepInDir(dirs); } } - private static TestShuffleDataContext initDataContext(boolean withShuffleFiles) - throws IOException { - if (withShuffleFiles) { - return initDataContextWithShuffleFiles(); - } else { - return initDataContextWithoutShuffleFiles(); - } - } - - private static TestShuffleDataContext initDataContextWithShuffleFiles() throws IOException { - TestShuffleDataContext dataContext = createDataContext(); - createShuffleFiles(dataContext); - createNonShuffleFiles(dataContext); - return dataContext; - } - - private static TestShuffleDataContext initDataContextWithoutShuffleFiles() throws IOException { - TestShuffleDataContext dataContext = createDataContext(); - createNonShuffleFiles(dataContext); - return dataContext; - } - - private static TestShuffleDataContext createDataContext() { + private static TestShuffleDataContext initDataContext(boolean withFilesToKeep) + throws IOException { TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); dataContext.create(); + if (withFilesToKeep) { + createFilesToKeep(dataContext); + } else { + createRemovableTestFiles(dataContext); + } return dataContext; } - private static void createShuffleFiles(TestShuffleDataContext dataContext) throws IOException { + private static void createFilesToKeep(TestShuffleDataContext dataContext) throws IOException { Random rand = new Random(123); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), "DEF".getBytes(StandardCharsets.UTF_8)}); + dataContext.insertCachedRddData(); } - private static void createNonShuffleFiles(TestShuffleDataContext dataContext) throws IOException { - // Create spill file(s) + private static void createRemovableTestFiles(TestShuffleDataContext dataContext) throws IOException { dataContext.insertSpillData(); + dataContext.insertBroadcastData(); + dataContext.insertTempShuffleData(); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 6989c3baf2e2..71fb10173891 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -98,8 +98,26 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr /** Creates spill file(s) within the local dirs. */ public void insertSpillData() throws IOException { String filename = "temp_local_" + UUID.randomUUID(); - OutputStream dataStream = null; + insertFile(filename); + } + public void insertBroadcastData() throws IOException { + String filename = "broadcast_12_" + UUID.randomUUID(); + insertFile(filename); + } + + public void insertTempShuffleData() throws IOException { + String filename = "temp_shuffle_" + UUID.randomUUID(); + insertFile(filename); + } + + public void insertCachedRddData() throws IOException { + String filename = "rdd_12_34"; + insertFile(filename); + } + + private void insertFile(String filename) throws IOException { + OutputStream dataStream = null; try { dataStream = new FileOutputStream( ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename)); diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 35ea887eb487..974e54689cc2 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -100,19 +100,10 @@ private[deploy] class Worker( // TTL for app folders/data; after TTL expires it will be cleaned up private val APP_DATA_RETENTION_SECONDS = conf.get(APP_DATA_RETENTION) - // Whether or not cleanup the non-shuffle files on executor exits. - private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = + // Whether or not cleanup the non-shuffle service served files on executor exits. + private val CLEANUP_FILES_AFTER_EXECUTOR_EXIT = conf.get(config.STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT) - val EXTERNAL_SHUFFLE_SERVICE_ENABLED = conf.get(config.SHUFFLE_SERVICE_ENABLED) - - if (CLEANUP_NON_SHUFFLE_FILES_ENABLED && EXTERNAL_SHUFFLE_SERVICE_ENABLED) { - logWarning("Both 'spark.storage.cleanupFilesAfterExecutorExit' and " + - "'spark.shuffle.service.enabled' are switched on. But with SPARK-25888 the external " + - "shuffle service able to serve disk persisted RDD blocks. So to keep benefiting from " + - "SPARK-25888 the cleanup only will be triggered after the application is stopped.") - } - private var master: Option[RpcEndpointRef] = None /** @@ -760,7 +751,7 @@ private[deploy] class Worker( coresUsed -= executor.cores memoryUsed -= executor.memory - if (CLEANUP_NON_SHUFFLE_FILES_ENABLED && !EXTERNAL_SHUFFLE_SERVICE_ENABLED) { + if (CLEANUP_FILES_AFTER_EXECUTOR_EXIT) { shuffleService.executorRemoved(executorStateChanged.execId.toString, appId) } case None => 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 0aed1af023f8..ca71e849d906 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 @@ -299,7 +299,7 @@ package object config { private[spark] val STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT = ConfigBuilder("spark.storage.cleanupFilesAfterExecutorExit") - .doc("Whether or not cleanup the non-shuffle files on executor exits.") + .doc("Whether or not cleanup the non-shuffle and non-RDD files on executor exits.") .booleanConf .createWithDefault(true) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 280c1208e4c1..e49a0848e49e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -449,7 +449,7 @@ class BlockManagerMasterEndpoint( locations.remove(blockManagerId) } - if (storageLevel.useDisk && externalShuffleServiceEnabled) { + if (blockId.isRDD && storageLevel.useDisk && externalShuffleServiceEnabled) { val externalShuffleServiceId = externalShuffleServiceIdOnHost(blockManagerId) if (storageLevel.isValid) { locations.add(externalShuffleServiceId) From 7f2fc12c4bb2f14c080eed3435d7b6ec4f0071d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Fri, 3 May 2019 18:34:49 +0200 Subject: [PATCH 10/25] fix line length --- .../shuffle/CleanupNonShuffleServiceServedFilesSuite.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index 7aab494af395..9be623072c86 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -196,7 +196,8 @@ private static void createFilesToKeep(TestShuffleDataContext dataContext) throws dataContext.insertCachedRddData(); } - private static void createRemovableTestFiles(TestShuffleDataContext dataContext) throws IOException { + private static void createRemovableTestFiles(TestShuffleDataContext dataContext) + throws IOException { dataContext.insertSpillData(); dataContext.insertBroadcastData(); dataContext.insertTempShuffleData(); From 6b08379435763ce5d96dad2cb250b45d9c8f50c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Mon, 6 May 2019 19:28:43 +0200 Subject: [PATCH 11/25] applying review comments --- .../shuffle/ExternalShuffleBlockHandler.java | 2 +- .../shuffle/ExternalShuffleBlockResolver.java | 5 ++-- .../ExternalShuffleBlockHandlerSuite.java | 16 +++++----- .../spark/storage/BlockManagerInfoSuite.scala | 30 +++++++++---------- 4 files changed, 25 insertions(+), 28 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index b4a23a9caf46..3ee82c20659f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -228,7 +228,7 @@ private class ManagedBufferIterator implements Iterator { } else if (blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { final int[] rddAndSplitIds = rddAndSplitIds(blockIds); size = rddAndSplitIds.length; - blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, + blockDataForIndexFn = index -> blockManager.getRddBlockData(appId, execId, rddAndSplitIds[index], rddAndSplitIds[index + 1]); } else { throw new IllegalArgumentException("Unexpected block id format: " + blockIds[0]); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 21827b17e24b..7f0c0f38ae90 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -179,7 +179,7 @@ public ManagedBuffer getBlockData( return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); } - public ManagedBuffer getBlockData( + public ManagedBuffer getRddBlockData( String appId, String execId, int rddId, @@ -313,10 +313,9 @@ public ManagedBuffer getDiskPersistedRddBlockData( ExecutorShuffleInfo executor, int rddId, int splitIndex) { File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, "rdd_" + rddId + "_" + splitIndex); - long length = file.length(); ManagedBuffer res = null; if (file.exists()) { - res = new FileSegmentManagedBuffer(conf, file, 0, length); + res = new FileSegmentManagedBuffer(conf, file, 0, file.length()); } return res; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 5d01b96f1ec4..d51e14a66faf 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -94,13 +94,13 @@ public void testOpenShuffleBlocks() { @Test public void testOpenDiskPersistedRDDBlocks() { - when(blockResolver.getBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); - when(blockResolver.getBlockData("app0", "exec1", 0, 1)).thenReturn(blockMarkers[1]); + when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getRddBlockData("app0", "exec1", 0, 1)).thenReturn(blockMarkers[1]); checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }, blockMarkers); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 1); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 1); verifyOpenBlockLatencyMetrics(); } @@ -110,15 +110,15 @@ public void testOpenDiskPersistedRDDBlocksWithMissingBlock() { new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), null }; - when(blockResolver.getBlockData("app0", "exec1", 0, 0)) + when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)) .thenReturn(blockMarkersWithMissingBlock[0]); - when(blockResolver.getBlockData("app0", "exec1", 0, 1)) + when(blockResolver.getRddBlockData("app0", "exec1", 0, 1)) .thenReturn(null); checkOpenBlocksReceive(new String[] { "rdd_0_0", "rdd_0_1" }, blockMarkersWithMissingBlock); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 1); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 1); } private void checkOpenBlocksReceive(String[] blockIds, ManagedBuffer[] blockMarkers) { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 7cb30c0b24b8..56ebebc44d8e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -48,7 +48,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { assert(bmInfo.blocks.asScala === Map(broadcastId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 100))) assert(bmInfo.exclusiveCachedBlocks.isEmpty) - assert(bmInfo.remainingMem == 29800) + assert(bmInfo.remainingMem === 29800) } testWithShuffleServiceOnOff("RDD block with MEMORY_ONLY") { (svcEnabled, bmInfo) => @@ -57,7 +57,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) assert(bmInfo.exclusiveCachedBlocks === Set(rddId)) - assert(bmInfo.remainingMem == 29800) + assert(bmInfo.remainingMem === 29800) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } @@ -71,7 +71,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(bmInfo.remainingMem == 29800) + assert(bmInfo.remainingMem === 29800) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) @@ -85,7 +85,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) @@ -97,16 +97,16 @@ class BlockManagerInfoSuite extends SparkFunSuite { bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) assert(bmInfo.exclusiveCachedBlocks === Set(rddId)) - assert(bmInfo.remainingMem == 29800) + assert(bmInfo.remainingMem === 29800) if (svcEnabled) { - assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala == Map()) + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) val exclusiveCachedBlocksForNoMemoryOnly = if (svcEnabled) Set() else Set(rddId) assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForNoMemoryOnly) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) @@ -116,11 +116,10 @@ class BlockManagerInfoSuite extends SparkFunSuite { testWithShuffleServiceOnOff("using invalid StorageLevel") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(bmInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) @@ -129,20 +128,19 @@ class BlockManagerInfoSuite extends SparkFunSuite { bmInfo.updateBlockInfo(rddId, StorageLevel.NONE, memSize = 0, diskSize = 200) assert(bmInfo.blocks.isEmpty) assert(bmInfo.exclusiveCachedBlocks.isEmpty) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { - assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map()) + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } } testWithShuffleServiceOnOff("remove block") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) - assert(bmInfo.blocks.asScala - === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) @@ -151,7 +149,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { bmInfo.removeBlock(rddId) assert(bmInfo.blocks.asScala.isEmpty) assert(bmInfo.exclusiveCachedBlocks.isEmpty) - assert(bmInfo.remainingMem == 30000) + assert(bmInfo.remainingMem === 30000) if (svcEnabled) { assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) } From 767f5f73a1afaafd9c9ebb6d279a6ab93cf6d8e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 7 May 2019 15:31:56 +0200 Subject: [PATCH 12/25] Applying review comments (from Imran) --- .../apache/spark/ExecutorAllocationManager.scala | 8 ++++---- .../apache/spark/storage/BlockManagerMaster.scala | 3 ++- .../storage/BlockManagerMasterEndpoint.scala | 4 ++-- .../spark/storage/BlockManagerInfoSuite.scala | 15 --------------- 4 files changed, 8 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 952f216c13c1..1782027fe293 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -491,7 +491,7 @@ private[spark] class ExecutorAllocationManager( newExecutorTotal = numExistingExecutors if (testing || executorsRemoved.nonEmpty) { executorsRemoved.foreach { removedExecutorId => - // If it is a cached block, it uses cachedExecutorIdleTimeoutS for timeout + // If it has an exclusive cached block then cachedExecutorIdleTimeoutS is used for timeout val idleTimeout = if (blockManagerMaster.hasExclusiveCachedBlocks(removedExecutorId)) { cachedExecutorIdleTimeoutS } else { @@ -605,9 +605,9 @@ private[spark] class ExecutorAllocationManager( private def onExecutorIdle(executorId: String): Unit = synchronized { if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { - // Note that it is not necessary to query the executors since all the cached - // blocks we are concerned with are reported to the driver. Note that this - // does not include broadcast blocks. + // Note that it is not necessary to query the executors since all the cached blocks we are + // concerned with are reported to the driver. This does not include broadcast blocks and + // non-exclusive blocks which are also available via the external shuffle service. val hasCachedBlocks = blockManagerMaster.hasExclusiveCachedBlocks(executorId) val now = clock.getTimeMillis() val timeout = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index ab5b89d346a0..bb776d196356 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -223,7 +223,8 @@ class BlockManagerMaster( } /** - * Find out if the executor has cached blocks which are only available via this executor. + * Find out if the executor has cached blocks which are not available via the external shuffle + * service. * This method does not consider broadcast blocks, since they are not reported the master. */ def hasExclusiveCachedBlocks(executorId: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index e49a0848e49e..5a21931307e3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -548,8 +548,8 @@ private[spark] class BlockManagerInfo( private val _blocks = new JHashMap[BlockId, BlockStatus] /** - * Cached blocks held exclusively by this BlockManager. This does not include broadcast blocks - * and local disk persisted blocks when external shuffle service is enabled. + * Cached blocks which are not available via the external shuffle service. + * This does not include broadcast blocks. */ private val _exclusiveCachedBlocks = new mutable.HashSet[BlockId] diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 56ebebc44d8e..29ae5dd52f9b 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -63,21 +63,6 @@ class BlockManagerInfoSuite extends SparkFunSuite { } } - testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => - val rddId: BlockId = RDDBlockId(0, 0) - bmInfo.updateBlockInfo( - rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) - assert(bmInfo.blocks.asScala === - Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) - val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) - assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) - assert(bmInfo.remainingMem === 29800) - if (svcEnabled) { - assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === - Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) - } - } - testWithShuffleServiceOnOff("RDD block with DISK_ONLY") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) From 2aa20972c310c8b2b849f9e73cece095431b46c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 7 May 2019 20:03:23 +0200 Subject: [PATCH 13/25] adding back MEMORY_AND_DISK --- .../spark/storage/BlockManagerInfoSuite.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 29ae5dd52f9b..6a61bdc8a1d9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -63,6 +63,20 @@ class BlockManagerInfoSuite extends SparkFunSuite { } } + testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.exclusiveCachedBlocks === exclusiveCachedBlocksForOneMemoryOnly) + assert(bmInfo.remainingMem === 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + } + } + testWithShuffleServiceOnOff("RDD block with DISK_ONLY") { (svcEnabled, bmInfo) => val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) From 928be2c7bbce41e3cb32b62284441fc223b3f7b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 8 May 2019 18:13:53 +0200 Subject: [PATCH 14/25] applying review comments of Imran v2.0 --- ...anupNonShuffleServiceServedFilesSuite.java | 4 +- .../ExternalShuffleIntegrationSuite.java | 73 +++++++++++++++++-- .../shuffle/TestShuffleDataContext.java | 13 ++-- .../spark/storage/BlockManagerMaster.scala | 2 +- .../storage/BlockManagerMasterEndpoint.scala | 2 + 5 files changed, 81 insertions(+), 13 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index 9be623072c86..7a0cf1575822 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -177,7 +177,7 @@ private static void assertCleanedUp(TestShuffleDataContext dataContext) { } private static TestShuffleDataContext initDataContext(boolean withFilesToKeep) - throws IOException { + throws IOException { TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); dataContext.create(); if (withFilesToKeep) { @@ -193,7 +193,7 @@ private static void createFilesToKeep(TestShuffleDataContext dataContext) throws dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), "DEF".getBytes(StandardCharsets.UTF_8)}); - dataContext.insertCachedRddData(); + dataContext.insertCachedRddData(12, 34, new byte[] { 42 }); } private static void createRemovableTestFiles(TestShuffleDataContext dataContext) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index f5b1ec9d46da..098ed13bf52f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -17,10 +17,12 @@ package org.apache.spark.network.shuffle; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -31,6 +33,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.server.OneForOneStreamManager; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -52,6 +56,11 @@ public class ExternalShuffleIntegrationSuite { private static final String APP_ID = "app-id"; private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + private static final int RDD_ID = 1; + private static final int SPLIT_INDEX_VALID_BLOCK = 0; + private static final int SPLIT_INDEX_MISSING_FILE = 1; + private static final int SPLIT_INDEX_CORRUPT_LENGTH = 2; + // Executor 0 is sort-based static TestShuffleDataContext dataContext0; @@ -60,6 +69,8 @@ public class ExternalShuffleIntegrationSuite { static TransportConf conf; static TransportContext transportContext; + static byte[] exec0RddBlock = new byte[123]; + static byte[][] exec0Blocks = new byte[][] { new byte[123], new byte[12345], @@ -81,13 +92,36 @@ public static void beforeAll() throws IOException { for (byte[] block: exec1Blocks) { rand.nextBytes(block); } + rand.nextBytes(exec0RddBlock); dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); dataContext0.insertSortShuffleData(0, 0, exec0Blocks); - - conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); - handler = new ExternalShuffleBlockHandler(conf, null); + dataContext0.insertCachedRddData( RDD_ID, SPLIT_INDEX_VALID_BLOCK, exec0RddBlock); + + HashMap config = new HashMap<>(); + config.put("spark.shuffle.io.maxRetries", "0"); + conf = new TransportConf("shuffle", new MapConfigProvider(config)); + handler = new ExternalShuffleBlockHandler( + new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, null) { + @Override + public ManagedBuffer getRddBlockData(String appId, String execId, int rddId, int splitIdx) { + ManagedBuffer res; + if (rddId == RDD_ID) { + switch (splitIdx) { + case SPLIT_INDEX_CORRUPT_LENGTH: + res = new FileSegmentManagedBuffer(conf, new File("missing.file"), 0, 12); + break; + default: + res = super.getRddBlockData(appId, execId, rddId, splitIdx); + } + } else { + res = super.getRddBlockData(appId, execId, rddId, splitIdx); + } + return res; + } + }); transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } @@ -199,9 +233,38 @@ public void testRegisterInvalidExecutor() throws Exception { @Test public void testFetchWrongBlockId() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" }); + FetchResult execFetch = fetchBlocks("exec-1", new String[] { "broadcast_1" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("broadcast_1"), execFetch.failedBlocks); + } + + @Test + public void testFetchValidRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String validBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_VALID_BLOCK; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { validBlockId }); + assertTrue(execFetch.failedBlocks.isEmpty()); + assertEquals(Sets.newHashSet(validBlockId), execFetch.successBlocks); + assertBuffersEqual(new NioManagedBuffer(ByteBuffer.wrap(exec0RddBlock)), + execFetch.buffers.get(0)); + } + + @Test + public void testFetchDeletedRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String missingBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_MISSING_FILE; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { missingBlockId }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet(missingBlockId), execFetch.failedBlocks); + } + + @Test + public void testFetchCorruptRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String corruptBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_CORRUPT_LENGTH; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { corruptBlockId }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet(corruptBlockId), execFetch.failedBlocks); } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 71fb10173891..f079946c76ab 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -111,17 +111,20 @@ public void insertTempShuffleData() throws IOException { insertFile(filename); } - public void insertCachedRddData() throws IOException { - String filename = "rdd_12_34"; - insertFile(filename); + public void insertCachedRddData(int rddId, int splitId, byte[] block) throws IOException { + String blockId = "rdd_" + rddId + "_" + splitId; + insertFile(blockId, block); } - private void insertFile(String filename) throws IOException { + insertFile(filename, new byte[] { 42 }); + } + + private void insertFile(String filename, byte[] block) throws IOException { OutputStream dataStream = null; try { dataStream = new FileOutputStream( ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename)); - dataStream.write(42); + dataStream.write(block); } finally { Closeables.close(dataStream, false); } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index bb776d196356..b18d38fe5253 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -225,7 +225,7 @@ class BlockManagerMaster( /** * Find out if the executor has cached blocks which are not available via the external shuffle * service. - * This method does not consider broadcast blocks, since they are not reported the master. + * This method does not consider broadcast blocks, since they are not reported to the master. */ def hasExclusiveCachedBlocks(executorId: String): Boolean = { driverEndpoint.askSync[Boolean](HasExclusiveCachedBlocks(executorId)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 5a21931307e3..eafde627b625 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -360,6 +360,8 @@ class BlockManagerMasterEndpoint( } private def externalShuffleServiceIdOnHost(blockManagerId: BlockManagerId): BlockManagerId = { + // we need to keep the executor ID of the original executor to let the shuffle service know + // which local directories should be used to look for the file BlockManagerId(blockManagerId.executorId, blockManagerId.host, externalShuffleServicePort) } From 79ed69af978755b83776b329133d54e56449431e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Mon, 13 May 2019 16:08:12 +0200 Subject: [PATCH 15/25] applying review comments of Imran v3.0 --- .../server/OneForOneStreamManagerSuite.java | 30 ++++++++++++++++--- .../spark/ExternalShuffleServiceSuite.scala | 4 +-- .../spark/storage/BlockManagerInfoSuite.scala | 4 +++ 3 files changed, 31 insertions(+), 7 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index a0ddea462210..fb3503b783e5 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -21,6 +21,7 @@ import java.util.List; import io.netty.channel.Channel; +import org.junit.After; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -30,6 +31,22 @@ public class OneForOneStreamManagerSuite { + List managedBuffersToRelease = new ArrayList<>(); + + @After + public void tearDown() { + managedBuffersToRelease.forEach(managedBuffer -> managedBuffer.release()); + managedBuffersToRelease.clear(); + } + + private ManagedBuffer getChunk(OneForOneStreamManager manager, long streamId, int chunkIndex) { + ManagedBuffer chunk = manager.getChunk(streamId, chunkIndex); + if (chunk != null) { + managedBuffersToRelease.add(chunk); + } + return chunk; + } + @Test public void testMissingChunk() { OneForOneStreamManager manager = new OneForOneStreamManager(); @@ -39,6 +56,8 @@ public void testMissingChunk() { TestManagedBuffer buffer3 = Mockito.spy(new TestManagedBuffer(20)); buffers.add(buffer1); + // the nulls here are to simulate a file which goes missing before being read, + // just as a defensive measure buffers.add(null); buffers.add(buffer2); buffers.add(null); @@ -47,17 +66,20 @@ public void testMissingChunk() { Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); long streamId = manager.registerStream("appId", buffers.iterator(), dummyChannel); Assert.assertEquals(1, manager.numStreamStates()); - Assert.assertNotNull(manager.getChunk(streamId, 0)); - Assert.assertNull(manager.getChunk(streamId, 1)); - Assert.assertNotNull(manager.getChunk(streamId, 2)); + Assert.assertNotNull(getChunk(manager, streamId, 0)); + Assert.assertNull(getChunk(manager, streamId, 1)); + Assert.assertNotNull(getChunk(manager, streamId, 2)); manager.connectionTerminated(dummyChannel); + + // loaded buffers are not released yet as in production a MangedBuffer returned by getChunk() + // would only be released by Netty after it is written to the network Mockito.verify(buffer1, Mockito.never()).release(); Mockito.verify(buffer2, Mockito.never()).release(); Mockito.verify(buffer3, Mockito.times(1)).release(); } @Test - public void managedBuffersAreFeedWhenConnectionIsClosed() { + public void managedBuffersAreFreedWhenConnectionIsClosed() { OneForOneStreamManager manager = new OneForOneStreamManager(); List buffers = new ArrayList<>(); TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index b4cb7fcb66d9..090731407bd9 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -127,8 +127,6 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi val rddSplit0Block = sc.env.blockManager.getRemoteValues(blockId) assert(rddSplit0Block.isDefined) - // Invalidate the registered executors, disallowing access to their shuffle blocks (without - // deleting the actual shuffle files, so we could access them without the shuffle service). - rpcHandler.applicationRemoved(sc.conf.getAppId, false /* cleanupLocalDirs */) + rpcHandler.applicationRemoved(sc.conf.getAppId, true) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 6a61bdc8a1d9..c951f663ac71 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -64,6 +64,8 @@ class BlockManagerInfoSuite extends SparkFunSuite { } testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => + // This is the effective storage level, not the requested storage level, but MEMORY_AND_DISK + // is still possible if its first in memory, purged to disk, and later promoted back to memory. val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) assert(bmInfo.blocks.asScala === @@ -92,6 +94,8 @@ class BlockManagerInfoSuite extends SparkFunSuite { } testWithShuffleServiceOnOff("update from MEMORY_ONLY to DISK_ONLY") { (svcEnabled, bmInfo) => + // This happens if MEMORY_AND_DISK is the requested storage level, but the block gets purged + // to disk under memory pressure. val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) From f8beff11b33800b726b3eeef0f8288439a5e29ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Thu, 16 May 2019 19:11:44 +0200 Subject: [PATCH 16/25] extend External Shuffle Service with remove blocks --- .../shuffle/ExternalShuffleBlockHandler.java | 17 ++-- .../shuffle/ExternalShuffleBlockResolver.java | 18 ++++ .../shuffle/ExternalShuffleClient.java | 44 ++++++++- .../spark/network/shuffle/ShuffleClient.java | 6 -- .../protocol/BlockTransferMessage.java | 5 +- .../shuffle/protocol/BlocksRemoved.java | 72 +++++++++++++++ .../shuffle/protocol/RemoveBlocks.java | 89 +++++++++++++++++++ .../ExternalShuffleIntegrationSuite.java | 31 ++++++- .../scala/org/apache/spark/SparkEnv.scala | 33 +++++-- .../apache/spark/storage/BlockManager.scala | 21 ++--- .../storage/BlockManagerMasterEndpoint.scala | 89 ++++++++++++++----- .../spark/ExternalShuffleServiceSuite.scala | 6 +- .../BlockManagerReplicationSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 18 ++-- 14 files changed, 386 insertions(+), 69 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 3ee82c20659f..ba9d657d0e56 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -100,11 +100,12 @@ protected void handleMessage( long streamId = streamManager.registerStream(client.getClientId(), new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds), client.getChannel()); if (logger.isTraceEnabled()) { - logger.trace("Registered streamId {} with {} buffers for client {} from host {}", - streamId, - msg.blockIds.length, - client.getClientId(), - getRemoteAddress(client.getChannel())); + logger.trace( + "Registered streamId {} with {} buffers for client {} from host {}", + streamId, + msg.blockIds.length, + client.getClientId(), + getRemoteAddress(client.getChannel())); } callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } finally { @@ -123,6 +124,12 @@ protected void handleMessage( responseDelayContext.stop(); } + } else if (msgObj instanceof RemoveBlocks) { + RemoveBlocks msg = (RemoveBlocks) msgObj; + checkAuth(client, msg.appId); + int numRemovedBlocks = blockManager.removeBlocks(msg.appId, msg.execId, msg.blockIds); + callback.onSuccess(new BlocksRemoved(numRemovedBlocks).toByteBuffer()); + } else { throw new UnsupportedOperationException("Unexpected message: " + msgObj); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 7f0c0f38ae90..8420eed524d2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -365,6 +365,24 @@ static String createNormalizedInternedPathname(String dir1, String dir2, String return pathname.intern(); } + public int removeBlocks(String appId, String execId, String[] blockIds) { + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + int numRemovedBlocks = 0; + for (String blockId : blockIds) { + File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); + if (file.delete()) { + numRemovedBlocks++; + } else { + logger.warn("Failed to delete existing block: " + file.getAbsolutePath()); + } + } + return numRemovedBlocks; + } + /** Simply encodes an executor's full ID, which is appId + execId. */ public static class AppExecId { public final String appId; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index e49e27ab5aa7..e6c292b6cd81 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -19,10 +19,17 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.shuffle.protocol.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,8 +40,6 @@ import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.util.TransportConf; /** @@ -73,7 +78,10 @@ protected void checkInit() { assert appId != null : "Called before init()"; } - @Override + /** + * Initializes the ShuffleClient, specifying this Executor's appId. + * Must be called before any other method on the ShuffleClient. + */ public void init(String appId) { this.appId = appId; TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true, true); @@ -139,12 +147,40 @@ public void registerWithShuffleServer( String execId, ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException { checkInit(); - try (TransportClient client = clientFactory.createUnmanagedClient(host, port)) { + try (TransportClient client = clientFactory.createClient(host, port)) { ByteBuffer registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteBuffer(); client.sendRpcSync(registerMessage, registrationTimeoutMs); } } + public Future removeBlocks( + String host, + int port, + String execId, + String[] blockIds) throws IOException, InterruptedException { + checkInit(); + CompletableFuture numRemovedBlocksFuture = new CompletableFuture<>(); + ByteBuffer removeBlocksMessage = new RemoveBlocks(appId, execId, blockIds).toByteBuffer(); + final TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(removeBlocksMessage, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); + numRemovedBlocksFuture.complete(((BlocksRemoved)msgObj).numRemovedBlocks); + client.close(); + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Error trying to remove RDD blocks " + Arrays.toString(blockIds) + + " via external shuffle service from executor: " + execId, e); + numRemovedBlocksFuture.complete(0); + client.close(); + } + }); + return numRemovedBlocksFuture; + } + @Override public void close() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index 62b99c40f61f..0be5cf5ad922 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -25,12 +25,6 @@ /** Provides an interface for reading shuffle files, either from an Executor or external service. */ public abstract class ShuffleClient implements Closeable { - /** - * Initializes the ShuffleClient, specifying this Executor's appId. - * Must be called before any other method on the ShuffleClient. - */ - public void init(String appId) { } - /** * Fetch a sequence of blocks from a remote node asynchronously, * diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index a68a297519b6..e09775644963 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -35,6 +35,7 @@ * shuffle service. It returns a StreamHandle. * - UploadBlock is only handled by the NettyBlockTransferService. * - RegisterExecutor is only handled by the external shuffle service. + * - RemoveBlocks is only handled by the external shuffle service. */ public abstract class BlockTransferMessage implements Encodable { protected abstract Type type(); @@ -42,7 +43,7 @@ public abstract class BlockTransferMessage implements Encodable { /** Preceding every serialized message is its type, which allows us to deserialize it. */ public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), - HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6); + HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8); private final byte id; @@ -68,6 +69,8 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 4: return RegisterDriver.decode(buf); case 5: return ShuffleServiceHeartbeat.decode(buf); case 6: return UploadBlockStream.decode(buf); + case 7: return RemoveBlocks.decode(buf); + case 8: return BlocksRemoved.decode(buf); default: throw new IllegalArgumentException("Unknown message type: " + type); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java new file mode 100644 index 000000000000..3f04443871b6 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java @@ -0,0 +1,72 @@ +/* + * 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.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** The reply to remove blocks giving back the number of removed blocks. */ +public class BlocksRemoved extends BlockTransferMessage { + public final int numRemovedBlocks; + + public BlocksRemoved(int numRemovedBlocks) { + this.numRemovedBlocks = numRemovedBlocks; + } + + @Override + protected Type type() { return Type.BLOCKS_REMOVED; } + + @Override + public int hashCode() { + return Objects.hashCode(numRemovedBlocks); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("numRemovedBlocks", numRemovedBlocks) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof BlocksRemoved) { + BlocksRemoved o = (BlocksRemoved) other; + return Objects.equal(numRemovedBlocks, o.numRemovedBlocks); + } + return false; + } + + @Override + public int encodedLength() { + return 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeInt(numRemovedBlocks); + } + + public static BlocksRemoved decode(ByteBuf buf) { + int numRemovedBlocks = buf.readInt(); + return new BlocksRemoved(numRemovedBlocks); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java new file mode 100644 index 000000000000..1c718d307753 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java @@ -0,0 +1,89 @@ +/* + * 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.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.spark.network.protocol.Encoders; + +import java.util.Arrays; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** Request to remove a set of blocks. */ +public class RemoveBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final String[] blockIds; + + public RemoveBlocks(String appId, String execId, String[] blockIds) { + this.appId = appId; + this.execId = execId; + this.blockIds = blockIds; + } + + @Override + protected Type type() { return Type.REMOVE_BLOCKS; } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("blockIds", Arrays.toString(blockIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof RemoveBlocks) { + RemoveBlocks o = (RemoveBlocks) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Arrays.equals(blockIds, o.blockIds); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + Encoders.StringArrays.encodedLength(blockIds); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + Encoders.StringArrays.encode(buf, blockIds); + } + + public static RemoveBlocks decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + String[] blockIds = Encoders.StringArrays.decode(buf); + return new RemoveBlocks(appId, execId, blockIds); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 098ed13bf52f..55eac27da6b0 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Random; import java.util.Set; +import java.util.concurrent.Future; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -60,6 +61,8 @@ public class ExternalShuffleIntegrationSuite { private static final int SPLIT_INDEX_VALID_BLOCK = 0; private static final int SPLIT_INDEX_MISSING_FILE = 1; private static final int SPLIT_INDEX_CORRUPT_LENGTH = 2; + private static final int SPLIT_INDEX_VALID_BLOCK_TO_RM = 3; + private static final int SPLIT_INDEX_MISSING_BLOCK_TO_RM = 4; // Executor 0 is sort-based static TestShuffleDataContext dataContext0; @@ -69,7 +72,8 @@ public class ExternalShuffleIntegrationSuite { static TransportConf conf; static TransportContext transportContext; - static byte[] exec0RddBlock = new byte[123]; + static byte[] exec0RddBlockValid = new byte[123]; + static byte[] exec0RddBlockToRemove = new byte[124]; static byte[][] exec0Blocks = new byte[][] { new byte[123], @@ -92,12 +96,14 @@ public static void beforeAll() throws IOException { for (byte[] block: exec1Blocks) { rand.nextBytes(block); } - rand.nextBytes(exec0RddBlock); + rand.nextBytes(exec0RddBlockValid); + rand.nextBytes(exec0RddBlockToRemove); dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); dataContext0.insertSortShuffleData(0, 0, exec0Blocks); - dataContext0.insertCachedRddData( RDD_ID, SPLIT_INDEX_VALID_BLOCK, exec0RddBlock); + dataContext0.insertCachedRddData(RDD_ID, SPLIT_INDEX_VALID_BLOCK, exec0RddBlockValid); + dataContext0.insertCachedRddData(RDD_ID, SPLIT_INDEX_VALID_BLOCK_TO_RM, exec0RddBlockToRemove); HashMap config = new HashMap<>(); config.put("spark.shuffle.io.maxRetries", "0"); @@ -245,7 +251,7 @@ public void testFetchValidRddBlock() throws Exception { FetchResult execFetch = fetchBlocks("exec-1", new String[] { validBlockId }); assertTrue(execFetch.failedBlocks.isEmpty()); assertEquals(Sets.newHashSet(validBlockId), execFetch.successBlocks); - assertBuffersEqual(new NioManagedBuffer(ByteBuffer.wrap(exec0RddBlock)), + assertBuffersEqual(new NioManagedBuffer(ByteBuffer.wrap(exec0RddBlockValid)), execFetch.buffers.get(0)); } @@ -258,6 +264,23 @@ public void testFetchDeletedRddBlock() throws Exception { assertEquals(Sets.newHashSet(missingBlockId), execFetch.failedBlocks); } + @Test + public void testRemoveRddBlocks() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String validBlockIdToRemove = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_VALID_BLOCK_TO_RM; + String missingBlockIdToRemove = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_MISSING_BLOCK_TO_RM; + + try (ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, 5000)) { + client.init(APP_ID); + Future numRemovedBlocks = client.removeBlocks( + TestUtils.getLocalHost(), + server.getPort(), + "exec-1", + new String[] { validBlockIdToRemove, missingBlockIdToRemove }); + assertEquals(1, numRemovedBlocks.get().intValue()); + } + } + @Test public void testFetchCorruptRddBlock() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 002bf65ba593..5a720eb59d6f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -35,7 +35,8 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} -import org.apache.spark.network.netty.NettyBlockTransferService +import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} +import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{LiveListenerBus, OutputCommitCoordinator} import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint @@ -328,9 +329,22 @@ object SparkEnv extends Logging { conf.get(BLOCK_MANAGER_PORT) } + val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) + Some(new ExternalShuffleClient(transConf, securityManager, + securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT))) + } else { + None + } + val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( BlockManagerMaster.DRIVER_ENDPOINT_NAME, - new BlockManagerMasterEndpoint(rpcEnv, isLocal, conf, listenerBus)), + new BlockManagerMasterEndpoint( + rpcEnv, + isLocal, + conf, + listenerBus, + externalShuffleClient)), conf, isDriver) val blockTransferService = @@ -338,9 +352,18 @@ object SparkEnv extends Logging { blockManagerPort, numUsableCores, blockManagerMaster.driverEndpoint) // NB: blockManager is not valid until initialize() is called later. - val blockManager = new BlockManager(executorId, rpcEnv, blockManagerMaster, - serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, - blockTransferService, securityManager, numUsableCores) + val blockManager = new BlockManager( + executorId, + rpcEnv, + blockManagerMaster, + serializerManager, + conf, + memoryManager, + mapOutputTracker, + shuffleManager, + blockTransferService, + securityManager, + externalShuffleClient) val metricsSystem = if (isDriver) { // Don't start metrics system right now for Driver. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8502f44fa2ce..6dbef784fe0c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -45,7 +45,6 @@ import org.apache.spark.metrics.source.Source import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.StreamCallbackWithID -import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle._ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.network.util.TransportConf @@ -130,11 +129,12 @@ private[spark] class BlockManager( shuffleManager: ShuffleManager, val blockTransferService: BlockTransferService, securityManager: SecurityManager, - numUsableCores: Int) + externalShuffleClient: Option[ExternalShuffleClient]) extends BlockDataManager with BlockEvictionHandler with Logging { - private[spark] val externalShuffleServiceEnabled = - conf.get(config.SHUFFLE_SERVICE_ENABLED) + // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)` + private[spark] val externalShuffleServiceEnabled: Boolean = externalShuffleClient.isDefined + private val remoteReadNioBufferConversion = conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION) @@ -174,13 +174,7 @@ private[spark] class BlockManager( // Client to read other executors' shuffle files. This is either an external service, or just the // standard BlockTransferService to directly connect to other Executors. - private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { - val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) - new ExternalShuffleClient(transConf, securityManager, - securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) - } else { - blockTransferService - } + private[spark] val shuffleClient = externalShuffleClient.getOrElse(blockTransferService) // Max number of failures before this block manager refreshes the block locations from the driver private val maxFailuresBeforeLocationRefresh = @@ -401,8 +395,9 @@ private[spark] class BlockManager( */ def initialize(appId: String): Unit = { blockTransferService.init(this) - shuffleClient.init(appId) - + externalShuffleClient.foreach { shuffleClient => + shuffleClient.init(appId) + } blockReplicationPolicy = { val priorityClass = conf.get(config.STORAGE_REPLICATION_POLICY) val clazz = Utils.classForName(priorityClass) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index eafde627b625..89444170e34f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -18,16 +18,22 @@ package org.apache.spark.storage import java.io.IOException +import java.nio.ByteBuffer import java.util.{HashMap => JHashMap} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.{ExecutionContext, Future} +import scala.collection.mutable.ListBuffer +import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Random import org.apache.spark.SparkConf + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.client.RpcResponseCallback +import org.apache.spark.network.shuffle.ExternalShuffleClient +import org.apache.spark.network.shuffle.protocol.{BlocksRemoved, BlockTransferMessage} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ @@ -42,7 +48,8 @@ class BlockManagerMasterEndpoint( override val rpcEnv: RpcEnv, val isLocal: Boolean, conf: SparkConf, - listenerBus: LiveListenerBus) + listenerBus: LiveListenerBus, + externalShuffleClient: Option[ExternalShuffleClient]) extends ThreadSafeRpcEndpoint with Logging { // Mapping from block manager id to the block manager's information. @@ -75,7 +82,8 @@ class BlockManagerMasterEndpoint( val proactivelyReplicate = conf.get(config.STORAGE_REPLICATION_PROACTIVE) logInfo("BlockManagerMasterEndpoint up") - private val externalShuffleServiceEnabled: Boolean = conf.get(config.SHUFFLE_SERVICE_ENABLED) + // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)` + private val externalShuffleServiceEnabled: Boolean = externalShuffleClient.isDefined private val externalShuffleServicePort: Int = StorageUtils.externalShuffleServicePort(conf) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -158,29 +166,68 @@ class BlockManagerMasterEndpoint( // First remove the metadata for the given RDD, and then asynchronously remove the blocks // from the slaves. + // The message sent to the slaves to remove the RDD + val removeMsg = RemoveRdd(rddId) + // Find all blocks for the given RDD, remove the block from both blockLocations and - // the blockManagerInfo that is tracking the blocks. + // the blockManagerInfo that is tracking the blocks and create the futures which asynchronously + // remove the blocks from those slaves which contains the blocks and gives back the number of + // removed blocks val blocks = blockLocations.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val removeRddFromExecutorsFutures = new ListBuffer[Future[Int]]() + val blocksToDeleteByShuffleService = + new mutable.HashMap[BlockManagerId, mutable.HashSet[RDDBlockId]] + blocks.foreach { blockId => - val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId) - bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) - blockLocations.remove(blockId) + val bms: mutable.HashSet[BlockManagerId] = blockLocations.remove(blockId) + + val (bmIdExtShuffle, bmIdExecutor) = bms.partition(_.port == externalShuffleServicePort) + if (bmIdExecutor.isEmpty && bmIdExtShuffle.nonEmpty) { + // when original executor is already removed use the shuffle service to remove the blocks + bmIdExtShuffle.foreach { bmIdForShuffleService => + val blockIdsToDel = blocksToDeleteByShuffleService.getOrElseUpdate(bmIdForShuffleService, + new mutable.HashSet[RDDBlockId]()) + blockIdsToDel += blockId + blockStatusByShuffleService.get(bmIdForShuffleService).foreach { blockStatus => + blockStatus.remove(blockId) + } + } + } else { + bmIdExecutor.foreach { bm => + blockManagerInfo.get(bm).foreach { bmInfo => + bmInfo.removeBlock(blockId) + removeRddFromExecutorsFutures += askRemoveRddFromExecutor(removeMsg, bmInfo) + } + } + } } - - // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. - // The dispatcher is used as an implicit argument into the Future sequence construction. - val removeMsg = RemoveRdd(rddId) - - val futures = blockManagerInfo.values.map { bm => - bm.slaveEndpoint.ask[Int](removeMsg).recover { - case e: IOException => - logWarning(s"Error trying to remove RDD $rddId from block manager ${bm.blockManagerId}", - e) - 0 // zero blocks were removed + val removeRddBlockViaExtShuffleServiceFutures = externalShuffleClient.map { shuffleClient => + blocksToDeleteByShuffleService.map { case (bmId, blockIds) => + Future[Int] { + shuffleClient.removeBlocks( + bmId.host, + bmId.port, + bmId.executorId, + blockIds.map(_.toString).toArray).get() + } } - }.toSeq + }.getOrElse(Seq.empty) - Future.sequence(futures) + Future.sequence(removeRddFromExecutorsFutures ++ removeRddBlockViaExtShuffleServiceFutures) + } + + /** + * Ask the slaves to remove the RDD. + */ + private def askRemoveRddFromExecutor( + removeMsg: RemoveRdd, + bmInfo: BlockManagerInfo): Future[Int] = { + bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { + case e: IOException => + logWarning(s"Error trying to remove RDD ${removeMsg.rddId} " + + s"from block manager ${bmInfo.blockManagerId}", e) + 0 // zero blocks were removed + } } private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { @@ -530,6 +577,8 @@ object BlockStatus { def empty: BlockStatus = BlockStatus(StorageLevel.NONE, memSize = 0L, diskSize = 0L) } + + private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 090731407bd9..db23d7f61091 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -124,9 +124,9 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi "external shuffle service port should be contained") } - val rddSplit0Block = sc.env.blockManager.getRemoteValues(blockId) - assert(rddSplit0Block.isDefined) - + assert(sc.env.blockManager.getRemoteValues(blockId).isDefined) + rdd.unpersist(true) + assert(sc.env.blockManager.getRemoteValues(blockId).isEmpty) rpcHandler.applicationRemoved(sc.conf.getAppId, true) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index a739701853f6..3f1a14fc91ed 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -75,7 +75,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val store = new BlockManager(name, rpcEnv, master, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, None) memManager.setMemoryStore(store.memoryStore) store.initialize("app-id") allStores += store @@ -99,7 +99,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) allStores.clear() } @@ -235,7 +235,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, None) memManager.setMemoryStore(failableStore.memoryStore) failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 731ad1a69a81..c8eb3442531e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -37,6 +37,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.config import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager @@ -45,7 +46,7 @@ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} -import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExternalShuffleClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus @@ -111,8 +112,15 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)) val memManager = UnifiedMemoryManager(bmConf, numCores = 1) val serializerManager = new SerializerManager(serializer, bmConf) + val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", 0) + Some(new ExternalShuffleClient(transConf, bmSecurityMgr, + bmSecurityMgr.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT))) + } else { + None + } val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, - memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, externalShuffleClient) memManager.setMemoryStore(blockManager.memoryStore) allStores += blockManager blockManager.initialize("app-id") @@ -135,7 +143,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(sc.conf).thenReturn(conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -894,7 +902,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) val store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, - shuffleManager, transfer, securityMgr, 0) + shuffleManager, transfer, securityMgr, None) allStores += store store.initialize("app-id") @@ -943,7 +951,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val blockManager = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, - shuffleManager, transfer, securityMgr, 0) + shuffleManager, transfer, securityMgr, None) try { blockManager.initialize("app-id") testPutBlockDataAsStream(blockManager, storageLevel) From d42eeeb6618419c10dc8afe6c5ce463649e3aeb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Thu, 16 May 2019 20:36:07 +0200 Subject: [PATCH 17/25] fix --- .../storage/BlockManagerMasterEndpoint.scala | 16 ++++++++-------- .../spark/ExternalShuffleServiceSuite.scala | 2 ++ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 89444170e34f..d003e8d1fbbc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -18,26 +18,23 @@ package org.apache.spark.storage import java.io.IOException -import java.nio.ByteBuffer import java.util.{HashMap => JHashMap} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ListBuffer -import scala.concurrent.{ExecutionContext, Future, Promise} +import scala.concurrent.{ExecutionContext, Future} import scala.util.Random import org.apache.spark.SparkConf - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} -import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.network.shuffle.ExternalShuffleClient -import org.apache.spark.network.shuffle.protocol.{BlocksRemoved, BlockTransferMessage} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} /** * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses @@ -81,6 +78,8 @@ class BlockManagerMasterEndpoint( val proactivelyReplicate = conf.get(config.STORAGE_REPLICATION_PROACTIVE) + val defaultRpcTimeout = RpcUtils.askRpcTimeout(conf) + logInfo("BlockManagerMasterEndpoint up") // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)` private val externalShuffleServiceEnabled: Boolean = externalShuffleClient.isDefined @@ -204,11 +203,12 @@ class BlockManagerMasterEndpoint( val removeRddBlockViaExtShuffleServiceFutures = externalShuffleClient.map { shuffleClient => blocksToDeleteByShuffleService.map { case (bmId, blockIds) => Future[Int] { - shuffleClient.removeBlocks( + val numRemovedBlocks = shuffleClient.removeBlocks( bmId.host, bmId.port, bmId.executorId, - blockIds.map(_.toString).toArray).get() + blockIds.map(_.toString).toArray) + numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds, TimeUnit.SECONDS) } } }.getOrElse(Seq.empty) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index db23d7f61091..84f0ce427307 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -125,6 +125,8 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi } assert(sc.env.blockManager.getRemoteValues(blockId).isDefined) + + // test unpersist: as executors are killed the blocks will be removed via the shuffle service rdd.unpersist(true) assert(sc.env.blockManager.getRemoteValues(blockId).isEmpty) rpcHandler.applicationRemoved(sc.conf.getAppId, true) From d6ca9c8f1fdc255882dc97840541e576642325f5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Thu, 16 May 2019 20:58:40 +0200 Subject: [PATCH 18/25] fix2 --- .../apache/spark/streaming/ReceivedBlockHandlerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 15060614983a..c8f424af9af0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -89,7 +89,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) @@ -282,7 +282,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) val memManager = new UnifiedMemoryManager(conf, maxMem, maxMem / 2, 1) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val blockManager = new BlockManager(name, rpcEnv, blockManagerMaster, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, None) memManager.setMemoryStore(blockManager.memoryStore) blockManager.initialize("app-id") blockManagerBuffer += blockManager From 39c991456871b975c489e3b61b48579eff686fa5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Fri, 17 May 2019 20:46:30 +0200 Subject: [PATCH 19/25] introduce spark.shuffle.service.fetch.rdd.enabled --- .../shuffle/ExternalShuffleBlockResolver.java | 18 ++- ...anupNonShuffleServiceServedFilesSuite.java | 123 +++++++++++++----- .../shuffle/TestShuffleDataContext.java | 7 +- .../scala/org/apache/spark/SparkEnv.scala | 6 +- .../spark/internal/config/package.scala | 12 +- .../storage/BlockManagerMasterEndpoint.scala | 11 +- 6 files changed, 125 insertions(+), 52 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 8420eed524d2..c777391cee05 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -86,6 +86,8 @@ public class ExternalShuffleBlockResolver { private final TransportConf conf; + private final boolean extShuffleServiceRddFetchEnabled; + @VisibleForTesting final File registeredExecutorFile; @VisibleForTesting @@ -109,6 +111,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF File registeredExecutorFile, Executor directoryCleaner) throws IOException { this.conf = conf; + this.extShuffleServiceRddFetchEnabled = + Boolean.valueOf(conf.get("spark.shuffle.service.fetch.rdd.enabled", "true")); this.registeredExecutorFile = registeredExecutorFile; String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); CacheLoader indexCacheLoader = @@ -245,7 +249,7 @@ public void executorRemoved(String executorId, String appId) { fullId, executor.localDirs.length); // Execute the actual deletion in a different thread, as it may take some time. - directoryCleaner.execute(() -> deleteNonShuffleNonRddFiles(executor.localDirs)); + directoryCleaner.execute(() -> deleteNonShuffleServiceServedFiles(executor.localDirs)); } } @@ -265,22 +269,24 @@ private void deleteExecutorDirs(String[] dirs) { } /** - * Synchronously deletes non-shuffle and non-RDD files in each directory recursively. + * Synchronously deletes files not served by shuffle service in each directory recursively. * Should be executed in its own thread, as this may take a long time. */ - private void deleteNonShuffleNonRddFiles(String[] dirs) { + private void deleteNonShuffleServiceServedFiles(String[] dirs) { FilenameFilter filter = (dir, name) -> { // Don't delete shuffle data, shuffle index files or cached RDD files. - return !name.endsWith(".index") && !name.endsWith(".data") && !name.startsWith("rdd_"); + return !name.endsWith(".index") && !name.endsWith(".data") + && (!extShuffleServiceRddFetchEnabled || !name.startsWith("rdd_")); }; for (String localDir : dirs) { try { JavaUtils.deleteRecursively(new File(localDir), filter); - logger.debug("Successfully cleaned up non-shuffle and non-RDD files in directory: {}", + logger.debug("Successfully cleaned up files not served by shuffle service in directory: {}", localDir); } catch (Exception e) { - logger.error("Failed to delete non-shuffle and non-RDD files in directory: " + localDir, e); + logger.error("Failed to delete files not served by shuffle service in directory: " + + localDir, e); } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index 7a0cf1575822..c47f0a722e43 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -18,15 +18,22 @@ package org.apache.spark.network.shuffle; import java.io.File; -import java.io.FilenameFilter; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Random; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.*; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.MoreExecutors; import org.junit.Test; + +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import org.apache.spark.network.util.MapConfigProvider; @@ -36,20 +43,42 @@ public class CleanupNonShuffleServiceServedFilesSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); - private TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + private static Set expectedShuffleFilesToKeep = + ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data"); + + private static Set expectedShuffleAndRddFilesToKeep = + ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data", "rdd_12_34"); + + private TransportConf getConf(boolean isFetchRddEnabled) { + return new TransportConf( + "shuffle", + new MapConfigProvider(ImmutableMap.of( + "spark.shuffle.service.fetch.rdd.enabled", + Boolean.toString(isFetchRddEnabled)))); + } + + @Test + public void cleanupOnRemovedExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnRemovedExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + @Test - public void cleanupOnRemovedExecutorWithFilesToKeep() throws IOException { - cleanupOnRemovedExecutor(true); + public void cleanupOnRemovedExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnRemovedExecutor(true, getConf(false), expectedShuffleFilesToKeep); } @Test public void cleanupOnRemovedExecutorWithoutFilesToKeep() throws IOException { - cleanupOnRemovedExecutor(false); + cleanupOnRemovedExecutor(false, getConf(true), Collections.emptySet()); } - private void cleanupOnRemovedExecutor(boolean withFilesToKeep) throws IOException { + private void cleanupOnRemovedExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); ExternalShuffleBlockResolver resolver = @@ -57,7 +86,7 @@ private void cleanupOnRemovedExecutor(boolean withFilesToKeep) throws IOExceptio resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); resolver.executorRemoved("exec0", "app"); - assertCleanedUp(dataContext); + assertContainedFilenames(dataContext, expectedFilesKept); } @Test @@ -79,7 +108,7 @@ private void cleanupUsesExecutor(boolean withFilesToKeep) throws IOException { Executor noThreadExecutor = runnable -> cleanupCalled.set(true); ExternalShuffleBlockResolver manager = - new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); + new ExternalShuffleBlockResolver(getConf(true), null, noThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); manager.executorRemoved("exec0", "app"); @@ -89,16 +118,24 @@ private void cleanupUsesExecutor(boolean withFilesToKeep) throws IOException { } @Test - public void cleanupOnlyRemovedExecutorWithFilesToKeep() throws IOException { - cleanupOnlyRemovedExecutor(true); + public void cleanupOnlyRemovedExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnlyRemovedExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + + @Test + public void cleanupOnlyRemovedExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnlyRemovedExecutor(true, getConf(false), expectedShuffleFilesToKeep); } @Test public void cleanupOnlyRemovedExecutorWithoutFilesToKeep() throws IOException { - cleanupOnlyRemovedExecutor(false); + cleanupOnlyRemovedExecutor(false, getConf(true) , Collections.emptySet()); } - private void cleanupOnlyRemovedExecutor(boolean withFilesToKeep) throws IOException { + private void cleanupOnlyRemovedExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { TestShuffleDataContext dataContext0 = initDataContext(withFilesToKeep); TestShuffleDataContext dataContext1 = initDataContext(withFilesToKeep); @@ -113,30 +150,38 @@ private void cleanupOnlyRemovedExecutor(boolean withFilesToKeep) throws IOExcept assertStillThere(dataContext1); resolver.executorRemoved("exec0", "app"); - assertCleanedUp(dataContext0); + assertContainedFilenames(dataContext0, expectedFilesKept); assertStillThere(dataContext1); resolver.executorRemoved("exec1", "app"); - assertCleanedUp(dataContext0); - assertCleanedUp(dataContext1); + assertContainedFilenames(dataContext0, expectedFilesKept); + assertContainedFilenames(dataContext1, expectedFilesKept); // Make sure it's not an error to cleanup multiple times resolver.executorRemoved("exec1", "app"); - assertCleanedUp(dataContext0); - assertCleanedUp(dataContext1); + assertContainedFilenames(dataContext0, expectedFilesKept); + assertContainedFilenames(dataContext1, expectedFilesKept); } @Test - public void cleanupOnlyRegisteredExecutorWithFilesToKeep() throws IOException { - cleanupOnlyRegisteredExecutor(true); + public void cleanupOnlyRegisteredExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnlyRegisteredExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + + @Test + public void cleanupOnlyRegisteredExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnlyRegisteredExecutor(true, getConf(false), expectedShuffleFilesToKeep); } @Test public void cleanupOnlyRegisteredExecutorWithoutFilesToKeep() throws IOException { - cleanupOnlyRegisteredExecutor(false); + cleanupOnlyRegisteredExecutor(false, getConf(true), Collections.emptySet()); } - private void cleanupOnlyRegisteredExecutor(boolean withFilesToKeep) throws IOException { + private void cleanupOnlyRegisteredExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); ExternalShuffleBlockResolver resolver = @@ -147,7 +192,7 @@ private void cleanupOnlyRegisteredExecutor(boolean withFilesToKeep) throws IOExc assertStillThere(dataContext); resolver.executorRemoved("exec0", "app"); - assertCleanedUp(dataContext); + assertContainedFilenames(dataContext, expectedFilesKept); } private static void assertStillThere(TestShuffleDataContext dataContext) { @@ -156,24 +201,32 @@ private static void assertStillThere(TestShuffleDataContext dataContext) { } } - private static FilenameFilter filter = (dir, name) -> { - // Don't delete shuffle data or shuffle index files. - return !name.endsWith(".index") && !name.endsWith(".data") && !name.startsWith("rdd_"); - }; - - private static boolean assertOnlyFilesToKeepInDir(File[] dirs) { - for (File dir : dirs) { - assertTrue(dir.getName() + " wasn't cleaned up", !dir.exists() || - dir.listFiles(filter).length == 0 || assertOnlyFilesToKeepInDir(dir.listFiles())); + private static Set collectFilenames(File[] files) { + Set result = new HashSet<>(); + for (File file : files) { + if (file.exists()) { + try (Stream walk = Files.walk(file.toPath())) { + result.addAll(walk + .filter(Files::isRegularFile) + .map(x -> x.toFile().getName()) + .collect(Collectors.toSet())); + } catch (IOException e) { + throw new AssertionError(e); + } + } } - return true; + return result; } - private static void assertCleanedUp(TestShuffleDataContext dataContext) { + private static void assertContainedFilenames( + TestShuffleDataContext dataContext, + Set expectedFilenames) { + Set collectedFilenames = new HashSet<>(); for (String localDir : dataContext.localDirs) { File[] dirs = new File[] {new File(localDir)}; - assertOnlyFilesToKeepInDir(dirs); + collectedFilenames.addAll(collectFilenames(dirs)); } + assertEquals(expectedFilenames, collectedFilenames); } private static TestShuffleDataContext initDataContext(boolean withFilesToKeep) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index f079946c76ab..78c7ec97152d 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -22,7 +22,6 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.util.UUID; import com.google.common.io.Closeables; import com.google.common.io.Files; @@ -97,17 +96,17 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr /** Creates spill file(s) within the local dirs. */ public void insertSpillData() throws IOException { - String filename = "temp_local_" + UUID.randomUUID(); + String filename = "temp_local_uuid"; insertFile(filename); } public void insertBroadcastData() throws IOException { - String filename = "broadcast_12_" + UUID.randomUUID(); + String filename = "broadcast_12_uuid"; insertFile(filename); } public void insertTempShuffleData() throws IOException { - String filename = "temp_shuffle_" + UUID.randomUUID(); + String filename = "temp_shuffle_uuid"; insertFile(filename); } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5a720eb59d6f..4e778a1ddd5f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -344,7 +344,11 @@ object SparkEnv extends Logging { isLocal, conf, listenerBus, - externalShuffleClient)), + if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { + externalShuffleClient + } else { + None + })), conf, isDriver) val blockTransferService = 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 ca71e849d906..1e4f1f344abb 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 @@ -299,7 +299,8 @@ package object config { private[spark] val STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT = ConfigBuilder("spark.storage.cleanupFilesAfterExecutorExit") - .doc("Whether or not cleanup the non-shuffle and non-RDD files on executor exits.") + .doc("Whether or not cleanup the files not served by the external shuffle service " + + "on executor exits.") .booleanConf .createWithDefault(true) @@ -366,6 +367,15 @@ package object config { private[spark] val SHUFFLE_SERVICE_ENABLED = ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) + private[spark] val SHUFFLE_SERVICE_FETCH_RDD_ENABLED = + ConfigBuilder("spark.shuffle.service.fetch.rdd.enabled") + .doc("Whether to use the ExternalShuffleService for fetching disk persisted RDD blocks. " + + "In case of dynamic allocation if this feature is enabled executors having only disk " + + "persisted blocks are considered idle after " + + "'spark.dynamicAllocation.executorIdleTimeout' and will be released accordingly.") + .booleanConf + .createWithDefault(true) + private[spark] val SHUFFLE_SERVICE_DB_ENABLED = ConfigBuilder("spark.shuffle.service.db.enabled") .doc("Whether to use db in ExternalShuffleService. Note that this only affects " + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index d003e8d1fbbc..d3be21246b70 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -81,8 +81,9 @@ class BlockManagerMasterEndpoint( val defaultRpcTimeout = RpcUtils.askRpcTimeout(conf) logInfo("BlockManagerMasterEndpoint up") - // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)` - private val externalShuffleServiceEnabled: Boolean = externalShuffleClient.isDefined + // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED) + // && conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)` + private val externalShuffleServiceRddFetchEnabled: Boolean = externalShuffleClient.isDefined private val externalShuffleServicePort: Int = StorageUtils.externalShuffleServicePort(conf) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -444,7 +445,7 @@ class BlockManagerMasterEndpoint( blockManagerIdByExecutor(id.executorId) = id val externalShuffleServiceBlockStatus = - if (externalShuffleServiceEnabled) { + if (externalShuffleServiceRddFetchEnabled) { val externalShuffleServiceBlocks = blockStatusByShuffleService .getOrElseUpdate(externalShuffleServiceIdOnHost(id), new JHashMap[BlockId, BlockStatus]) Some(externalShuffleServiceBlocks) @@ -498,7 +499,7 @@ class BlockManagerMasterEndpoint( locations.remove(blockManagerId) } - if (blockId.isRDD && storageLevel.useDisk && externalShuffleServiceEnabled) { + if (blockId.isRDD && storageLevel.useDisk && externalShuffleServiceRddFetchEnabled) { val externalShuffleServiceId = externalShuffleServiceIdOnHost(blockManagerId) if (storageLevel.isValid) { locations.add(externalShuffleServiceId) @@ -521,7 +522,7 @@ class BlockManagerMasterEndpoint( private def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = { val locations = Option(blockLocations.get(blockId)).map(_.toSeq).getOrElse(Seq.empty) val status = locations.headOption.flatMap { bmId => - if (externalShuffleServiceEnabled && bmId.port == externalShuffleServicePort) { + if (externalShuffleServiceRddFetchEnabled && bmId.port == externalShuffleServicePort) { Option(blockStatusByShuffleService(bmId).get(blockId)) } else { blockManagerInfo(bmId).getStatus(blockId) From 16ab64f65ae90344c4fcb30bb1a695f1fb0ffabd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Fri, 17 May 2019 23:10:18 +0200 Subject: [PATCH 20/25] fix checkstyle --- .../org/apache/spark/network/shuffle/ExternalShuffleClient.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index e6c292b6cd81..0e11d2124ada 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -22,13 +22,11 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.shuffle.protocol.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 43fac8bb85f735120b55f0bbb42effdd94fb205d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Sat, 18 May 2019 05:35:43 +0200 Subject: [PATCH 21/25] keep the old logic: sending RemoveRdd to every live executor --- .../network/shuffle/ExternalShuffleBlockResolver.java | 2 +- .../spark/storage/BlockManagerMasterEndpoint.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index c777391cee05..7fc7354ca770 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -383,7 +383,7 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { if (file.delete()) { numRemovedBlocks++; } else { - logger.warn("Failed to delete existing block: " + file.getAbsolutePath()); + logger.warn("Failed to delete block: " + file.getAbsolutePath()); } } return numRemovedBlocks; diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index d3be21246b70..6677a2e988ab 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ListBuffer import scala.concurrent.{ExecutionContext, Future} import scala.util.Random @@ -171,10 +170,8 @@ class BlockManagerMasterEndpoint( // Find all blocks for the given RDD, remove the block from both blockLocations and // the blockManagerInfo that is tracking the blocks and create the futures which asynchronously - // remove the blocks from those slaves which contains the blocks and gives back the number of - // removed blocks + // remove the blocks from slaves and gives back the number of removed blocks val blocks = blockLocations.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) - val removeRddFromExecutorsFutures = new ListBuffer[Future[Int]]() val blocksToDeleteByShuffleService = new mutable.HashMap[BlockManagerId, mutable.HashSet[RDDBlockId]] @@ -196,11 +193,14 @@ class BlockManagerMasterEndpoint( bmIdExecutor.foreach { bm => blockManagerInfo.get(bm).foreach { bmInfo => bmInfo.removeBlock(blockId) - removeRddFromExecutorsFutures += askRemoveRddFromExecutor(removeMsg, bmInfo) } } } } + val removeRddFromExecutorsFutures = blockManagerInfo.values.map { bmInfo => + askRemoveRddFromExecutor(removeMsg, bmInfo) + }.toSeq + val removeRddBlockViaExtShuffleServiceFutures = externalShuffleClient.map { shuffleClient => blocksToDeleteByShuffleService.map { case (bmId, blockIds) => Future[Int] { From 5a1a15aebe30fbb0c53a2c88c97d47cf4ed40bce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Sat, 18 May 2019 09:22:08 +0200 Subject: [PATCH 22/25] javastyle fix --- .../shuffle/CleanupNonShuffleServiceServedFilesSuite.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index c47f0a722e43..d024dcc2d341 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -201,7 +201,7 @@ private static void assertStillThere(TestShuffleDataContext dataContext) { } } - private static Set collectFilenames(File[] files) { + private static Set collectFilenames(File[] files) throws IOException { Set result = new HashSet<>(); for (File file : files) { if (file.exists()) { @@ -210,8 +210,6 @@ private static Set collectFilenames(File[] files) { .filter(Files::isRegularFile) .map(x -> x.toFile().getName()) .collect(Collectors.toSet())); - } catch (IOException e) { - throw new AssertionError(e); } } } @@ -220,7 +218,7 @@ private static Set collectFilenames(File[] files) { private static void assertContainedFilenames( TestShuffleDataContext dataContext, - Set expectedFilenames) { + Set expectedFilenames) throws IOException { Set collectedFilenames = new HashSet<>(); for (String localDir : dataContext.localDirs) { File[] dirs = new File[] {new File(localDir)}; From e3adc05edc625e7afcfb73710c46162247f7a812 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 21 May 2019 16:50:10 +0200 Subject: [PATCH 23/25] applying review comments of Vanzin --- .../spark/network/shuffle/Constant.java | 24 +++++++++++ .../shuffle/ExternalShuffleBlockResolver.java | 8 ++-- ...anupNonShuffleServiceServedFilesSuite.java | 8 ++-- .../shuffle/TestShuffleDataContext.java | 6 ++- .../spark/internal/config/package.scala | 3 +- .../storage/BlockManagerMasterEndpoint.scala | 42 +++++++------------ 6 files changed, 54 insertions(+), 37 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java new file mode 100644 index 000000000000..2d58449c35e3 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java @@ -0,0 +1,24 @@ +/* + * 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.network.shuffle; + +public class Constant { + + public static final String SHUFFLE_SERVICE_FETCH_RDD_ENABLED = + "spark.shuffle.service.fetch.rdd.enabled"; +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 7fc7354ca770..749be0934998 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -86,7 +86,7 @@ public class ExternalShuffleBlockResolver { private final TransportConf conf; - private final boolean extShuffleServiceRddFetchEnabled; + private final boolean rddFetchEnabled; @VisibleForTesting final File registeredExecutorFile; @@ -111,8 +111,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF File registeredExecutorFile, Executor directoryCleaner) throws IOException { this.conf = conf; - this.extShuffleServiceRddFetchEnabled = - Boolean.valueOf(conf.get("spark.shuffle.service.fetch.rdd.enabled", "true")); + this.rddFetchEnabled = + Boolean.valueOf(conf.get(Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, "true")); this.registeredExecutorFile = registeredExecutorFile; String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); CacheLoader indexCacheLoader = @@ -276,7 +276,7 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { FilenameFilter filter = (dir, name) -> { // Don't delete shuffle data, shuffle index files or cached RDD files. return !name.endsWith(".index") && !name.endsWith(".data") - && (!extShuffleServiceRddFetchEnabled || !name.startsWith("rdd_")); + && (!rddFetchEnabled || !name.startsWith("rdd_")); }; for (String localDir : dirs) { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index d024dcc2d341..06c70c38889e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -56,7 +56,7 @@ private TransportConf getConf(boolean isFetchRddEnabled) { return new TransportConf( "shuffle", new MapConfigProvider(ImmutableMap.of( - "spark.shuffle.service.fetch.rdd.enabled", + Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, Boolean.toString(isFetchRddEnabled)))); } @@ -104,11 +104,11 @@ private void cleanupUsesExecutor(boolean withFilesToKeep) throws IOException { AtomicBoolean cleanupCalled = new AtomicBoolean(false); - // Executor which does nothing to ensure we're actually using it. - Executor noThreadExecutor = runnable -> cleanupCalled.set(true); + // Executor which only captures whether it's being used, without executing anything. + Executor dummyExecutor = runnable -> cleanupCalled.set(true); ExternalShuffleBlockResolver manager = - new ExternalShuffleBlockResolver(getConf(true), null, noThreadExecutor); + new ExternalShuffleBlockResolver(getConf(true), null, dummyExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); manager.executorRemoved("exec0", "app"); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 78c7ec97152d..10be95ec50c3 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -114,15 +114,17 @@ public void insertCachedRddData(int rddId, int splitId, byte[] block) throws IOE String blockId = "rdd_" + rddId + "_" + splitId; insertFile(blockId, block); } + private void insertFile(String filename) throws IOException { insertFile(filename, new byte[] { 42 }); } private void insertFile(String filename, byte[] block) throws IOException { OutputStream dataStream = null; + File file = ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename); + assert(!file.exists()) : "this test file has been already generated"; try { - dataStream = new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename)); + dataStream = new FileOutputStream(file); dataStream.write(block); } finally { Closeables.close(dataStream, false); 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 1e4f1f344abb..0cfd7de94ad1 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 @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher import org.apache.spark.metrics.GarbageCollectionMetrics +import org.apache.spark.network.shuffle.Constant import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} @@ -368,7 +369,7 @@ package object config { ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) private[spark] val SHUFFLE_SERVICE_FETCH_RDD_ENABLED = - ConfigBuilder("spark.shuffle.service.fetch.rdd.enabled") + ConfigBuilder(Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED) .doc("Whether to use the ExternalShuffleService for fetching disk persisted RDD blocks. " + "In case of dynamic allocation if this feature is enabled executors having only disk " + "persisted blocks are considered idle after " + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 6677a2e988ab..65ec1c3f0dc6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -178,10 +178,12 @@ class BlockManagerMasterEndpoint( blocks.foreach { blockId => val bms: mutable.HashSet[BlockManagerId] = blockLocations.remove(blockId) - val (bmIdExtShuffle, bmIdExecutor) = bms.partition(_.port == externalShuffleServicePort) - if (bmIdExecutor.isEmpty && bmIdExtShuffle.nonEmpty) { - // when original executor is already removed use the shuffle service to remove the blocks - bmIdExtShuffle.foreach { bmIdForShuffleService => + val (bmIdsExtShuffle, bmIdsExecutor) = bms.partition(_.port == externalShuffleServicePort) + val liveExecutorsForBlock = bmIdsExecutor.map(_.executorId).toSet + bmIdsExtShuffle.foreach { bmIdForShuffleService => + // if the original executor is already released then delete this disk block via + // the external shuffle service + if (!liveExecutorsForBlock.contains(bmIdForShuffleService.executorId)) { val blockIdsToDel = blocksToDeleteByShuffleService.getOrElseUpdate(bmIdForShuffleService, new mutable.HashSet[RDDBlockId]()) blockIdsToDel += blockId @@ -189,16 +191,20 @@ class BlockManagerMasterEndpoint( blockStatus.remove(blockId) } } - } else { - bmIdExecutor.foreach { bm => - blockManagerInfo.get(bm).foreach { bmInfo => - bmInfo.removeBlock(blockId) - } + } + bmIdsExecutor.foreach { bmId => + blockManagerInfo.get(bmId).foreach { bmInfo => + bmInfo.removeBlock(blockId) } } } val removeRddFromExecutorsFutures = blockManagerInfo.values.map { bmInfo => - askRemoveRddFromExecutor(removeMsg, bmInfo) + bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { + case e: IOException => + logWarning(s"Error trying to remove RDD ${removeMsg.rddId} " + + s"from block manager ${bmInfo.blockManagerId}", e) + 0 // zero blocks were removed + } }.toSeq val removeRddBlockViaExtShuffleServiceFutures = externalShuffleClient.map { shuffleClient => @@ -217,20 +223,6 @@ class BlockManagerMasterEndpoint( Future.sequence(removeRddFromExecutorsFutures ++ removeRddBlockViaExtShuffleServiceFutures) } - /** - * Ask the slaves to remove the RDD. - */ - private def askRemoveRddFromExecutor( - removeMsg: RemoveRdd, - bmInfo: BlockManagerInfo): Future[Int] = { - bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { - case e: IOException => - logWarning(s"Error trying to remove RDD ${removeMsg.rddId} " + - s"from block manager ${bmInfo.blockManagerId}", e) - 0 // zero blocks were removed - } - } - private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { // Nothing to do in the BlockManagerMasterEndpoint data structures val removeMsg = RemoveShuffle(shuffleId) @@ -578,8 +570,6 @@ object BlockStatus { def empty: BlockStatus = BlockStatus(StorageLevel.NONE, memSize = 0L, diskSize = 0L) } - - private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, From bf9ec92265b133e1efdf7a6480a8d8ef6640f611 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 22 May 2019 11:29:48 +0200 Subject: [PATCH 24/25] applying review comments --- .../spark/network/shuffle/{Constant.java => Constants.java} | 2 +- .../spark/network/shuffle/ExternalShuffleBlockResolver.java | 2 +- .../shuffle/CleanupNonShuffleServiceServedFilesSuite.java | 4 ++-- .../main/scala/org/apache/spark/internal/config/package.scala | 4 ++-- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 3 +-- 5 files changed, 7 insertions(+), 8 deletions(-) rename common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/{Constant.java => Constants.java} (97%) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java similarity index 97% rename from common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java index 2d58449c35e3..01aca7efb12b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constant.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java @@ -17,7 +17,7 @@ package org.apache.spark.network.shuffle; -public class Constant { +public class Constants { public static final String SHUFFLE_SERVICE_FETCH_RDD_ENABLED = "spark.shuffle.service.fetch.rdd.enabled"; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 749be0934998..2d0bf40e254f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -112,7 +112,7 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF Executor directoryCleaner) throws IOException { this.conf = conf; this.rddFetchEnabled = - Boolean.valueOf(conf.get(Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, "true")); + Boolean.valueOf(conf.get(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, "true")); this.registeredExecutorFile = registeredExecutorFile; String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); CacheLoader indexCacheLoader = diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index 06c70c38889e..c2ac811519cd 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -56,7 +56,7 @@ private TransportConf getConf(boolean isFetchRddEnabled) { return new TransportConf( "shuffle", new MapConfigProvider(ImmutableMap.of( - Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, + Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, Boolean.toString(isFetchRddEnabled)))); } @@ -248,7 +248,7 @@ private static void createFilesToKeep(TestShuffleDataContext dataContext) throws } private static void createRemovableTestFiles(TestShuffleDataContext dataContext) - throws IOException { + throws IOException { dataContext.insertSpillData(); dataContext.insertBroadcastData(); dataContext.insertTempShuffleData(); 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 0cfd7de94ad1..882de1deb5e6 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 @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher import org.apache.spark.metrics.GarbageCollectionMetrics -import org.apache.spark.network.shuffle.Constant +import org.apache.spark.network.shuffle.Constants import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} @@ -369,7 +369,7 @@ package object config { ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) private[spark] val SHUFFLE_SERVICE_FETCH_RDD_ENABLED = - ConfigBuilder(Constant.SHUFFLE_SERVICE_FETCH_RDD_ENABLED) + ConfigBuilder(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED) .doc("Whether to use the ExternalShuffleService for fetching disk persisted RDD blocks. " + "In case of dynamic allocation if this feature is enabled executors having only disk " + "persisted blocks are considered idle after " + diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c8eb3442531e..59d58edc9dfe 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1379,8 +1379,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE new FileSegmentManagedBuffer(transConf, new File("missing.file"), 0, 0) } } - val store2 = - makeBlockManager(8000, "executor2", this.master, Some(emptyBlockFetcher)) + val store2 = makeBlockManager(8000, "executor2", this.master, Some(emptyBlockFetcher)) store.putSingle("item", "value", StorageLevel.DISK_ONLY, tellMaster = true) assert(master.getLocations("item").nonEmpty) assert(store2.getRemoteBytes("item").isEmpty) From faa583f88b410fc7ededafe36bcb1ef878482d44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Thu, 23 May 2019 13:26:30 +0200 Subject: [PATCH 25/25] fixing: test NITs --- .../shuffle/ExternalShuffleBlockResolver.java | 3 +- ...anupNonShuffleServiceServedFilesSuite.java | 2 +- .../spark/ExternalShuffleServiceSuite.scala | 56 ++++++++++--------- .../spark/storage/BlockManagerInfoSuite.scala | 2 +- 4 files changed, 33 insertions(+), 30 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 2d0bf40e254f..6b6ca9243b62 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -319,9 +319,10 @@ public ManagedBuffer getDiskPersistedRddBlockData( ExecutorShuffleInfo executor, int rddId, int splitIndex) { File file = getFile(executor.localDirs, executor.subDirsPerLocalDir, "rdd_" + rddId + "_" + splitIndex); + long fileLength = file.length(); ManagedBuffer res = null; if (file.exists()) { - res = new FileSegmentManagedBuffer(conf, file, 0, file.length()); + res = new FileSegmentManagedBuffer(conf, file, 0, fileLength); } return res; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index c2ac811519cd..e38442327e22 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -221,7 +221,7 @@ private static void assertContainedFilenames( Set expectedFilenames) throws IOException { Set collectedFilenames = new HashSet<>(); for (String localDir : dataContext.localDirs) { - File[] dirs = new File[] {new File(localDir)}; + File[] dirs = new File[] { new File(localDir) }; collectedFilenames.addAll(collectFilenames(dirs)); } assertEquals(expectedFilenames, collectedFilenames); diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 84f0ce427307..e5644f25a0b7 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -100,35 +100,37 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) sc.env.blockManager.externalShuffleServiceEnabled should equal(true) sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) + try { + val rdd = sc.parallelize(0 until 100, 2) + .map { i => (i, 1) } + .persist(StorageLevel.DISK_ONLY) - val rdd = sc.parallelize(0 until 100, 2) - .map { i => (i, 1) } - .persist(StorageLevel.DISK_ONLY) - - rdd.count() - - val blockId = RDDBlockId(rdd.id, 0) - eventually(timeout(2.seconds), interval(100.milliseconds)) { - val locations = sc.env.blockManager.master.getLocations(blockId) - assert(locations.size === 2) - assert(locations.map(_.port).contains(server.getPort), - "external shuffle service port should be contained") - } - - sc.killExecutors(sc.getExecutorIds()) + rdd.count() - eventually(timeout(2.seconds), interval(100.milliseconds)) { - val locations = sc.env.blockManager.master.getLocations(blockId) - assert(locations.size === 1) - assert(locations.map(_.port).contains(server.getPort), - "external shuffle service port should be contained") + val blockId = RDDBlockId(rdd.id, 0) + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 2) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + sc.killExecutors(sc.getExecutorIds()) + + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 1) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + assert(sc.env.blockManager.getRemoteValues(blockId).isDefined) + + // test unpersist: as executors are killed the blocks will be removed via the shuffle service + rdd.unpersist(true) + assert(sc.env.blockManager.getRemoteValues(blockId).isEmpty) + } finally { + rpcHandler.applicationRemoved(sc.conf.getAppId, true) } - - assert(sc.env.blockManager.getRemoteValues(blockId).isDefined) - - // test unpersist: as executors are killed the blocks will be removed via the shuffle service - rdd.unpersist(true) - assert(sc.env.blockManager.getRemoteValues(blockId).isEmpty) - rpcHandler.applicationRemoved(sc.conf.getAppId, true) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index c951f663ac71..8df123250303 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -65,7 +65,7 @@ class BlockManagerInfoSuite extends SparkFunSuite { testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => // This is the effective storage level, not the requested storage level, but MEMORY_AND_DISK - // is still possible if its first in memory, purged to disk, and later promoted back to memory. + // is still possible if it's first in memory, purged to disk, and later promoted back to memory. val rddId: BlockId = RDDBlockId(0, 0) bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) assert(bmInfo.blocks.asScala ===