From f144936012b4b6192814785abd6725f4595bb14c Mon Sep 17 00:00:00 2001 From: Min Shen Date: Sat, 9 May 2020 16:08:41 -0700 Subject: [PATCH 01/29] LIHADOOP-48527 Magnet shuffle service block transfer netty protocol The following changes are included in this patch. In addition, fixed a potential block duplicate issue when speculative execution is enabled, and improved test coverage. commit 7e134c2b75c8882474a67c15036087eb8a02caef Author: Chandni Singh Date: Tue Apr 21 21:54:46 2020 -0700 LIHADOOP-52202 Create merge_manager under app local dirs RB=2069854 G=superfriends-reviewers R=mshen,yezhou A=chsingh commit 63dcda9309fc06c5f1fb6e7268df1d7416db49c7 Author: Chandni Singh Date: Wed Apr 1 12:15:21 2020 -0700 LIHADOOP-51889 Divide remote fetches into smaller chunks RB=2029681 commit 35298465155ec10e6ee2caf1adc0e78717dc6fed Author: Chandni Singh Date: Thu Mar 19 17:47:40 2020 -0700 LIHADOOP-51889 Writing last chunk offsets to merge index file RB=2016700 BUG=LIHADOOP-51889 G=superfriends-reviewers R=mshen A=mshen commit bbb53ec0fdfa0aebda954ede17a9f6e217607a53 Author: Min Shen Date: Thu Dec 19 08:46:34 2019 -0800 Shuffle server and client properly handles merged block fetch failure. Use file length as merged shuffle block size when serving merged shuffle block. commit 6718074c6a6a98b1d66d4fdff6bf08fb266ce32e Author: Min Shen Date: Mon Nov 18 14:19:20 2019 -0800 Netty protocol for DAGScheduler control message commit 52e4dfade2e004fbc39fc60937342a9a57872680 Author: Min Shen Date: Sun Sep 8 18:44:09 2019 -0700 Netty protocol for remote block push, pass 3 commit e9db4cc1ae56e9722a598b0011a10e55e84bf19c Author: Min Shen Date: Thu Sep 5 18:29:24 2019 -0700 Netty protocol for remote block push, pass 2 commit 7627ecf54292edda4a133e596f53306e7af76100 Author: Min Shen Date: Fri Aug 30 08:54:08 2019 -0700 Netty protocol for remote block push, pass 1 --- common/network-common/pom.xml | 4 + .../spark/network/protocol/Encoders.java | 99 +++ .../server/TransportRequestHandler.java | 56 +- .../apache/spark/network/util/JavaUtils.java | 12 + .../spark/network/util/TransportConf.java | 12 + .../spark/network/RpcIntegrationSuite.java | 37 +- common/network-shuffle/pom.xml | 9 + .../network/shuffle/BlockPushException.java | 69 ++ .../network/shuffle/BlockStoreClient.java | 36 + .../network/shuffle/ExternalBlockHandler.java | 213 +++++- .../shuffle/ExternalBlockStoreClient.java | 109 +++ .../shuffle/MergeFinalizerListener.java | 35 + .../shuffle/MergedBlocksMetaListener.java | 41 ++ .../shuffle/MergedShuffleFileManager.java | 103 +++ .../shuffle/OneForOneBlockFetcher.java | 103 ++- .../network/shuffle/OneForOneBlockPusher.java | 125 ++++ .../shuffle/RemoteBlockPushResolver.java | 685 ++++++++++++++++++ .../protocol/AbstractFetchShuffleBlocks.java | 86 +++ .../protocol/BlockTransferMessage.java | 11 +- .../protocol/FetchMergedBlocksMeta.java | 83 +++ .../protocol/FetchShuffleBlockChunks.java | 123 ++++ .../shuffle/protocol/FetchShuffleBlocks.java | 43 +- .../protocol/FinalizeShuffleMerge.java | 85 +++ .../shuffle/protocol/MergeStatuses.java | 119 +++ .../shuffle/protocol/MergedBlocksMeta.java | 75 ++ .../shuffle/protocol/PushBlockStream.java | 92 +++ .../shuffle/ExternalBlockHandlerSuite.java | 150 +++- .../shuffle/OneForOneBlockFetcherSuite.java | 22 + .../shuffle/OneForOneBlockPusherSuite.java | 157 ++++ .../shuffle/RemoteBlockPushResolverSuite.java | 152 ++++ .../FetchShuffleBlockChunksSuite.java | 42 ++ .../protocol/FetchShuffleBlocksSuite.java | 42 ++ .../network/yarn/YarnShuffleService.java | 19 +- .../ExternalShuffleServiceMetricsSuite.scala | 4 +- .../yarn/YarnShuffleServiceMetricsSuite.scala | 3 +- 35 files changed, 2990 insertions(+), 66 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 9d5bc9aae0719..d328a7de0a762 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -91,6 +91,10 @@ org.apache.commons commons-crypto + + org.roaringbitmap + RoaringBitmap + diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 490915f6de4b3..b2111b2f3e237 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -17,9 +17,16 @@ package org.apache.spark.network.protocol; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import io.netty.buffer.ByteBuf; +import org.roaringbitmap.RoaringBitmap; /** Provides a canonical set of Encoders for simple types. */ public class Encoders { @@ -44,6 +51,71 @@ public static String decode(ByteBuf buf) { } } + /** Bitmaps are encoded with their serialization length followed by the serialization bytes. */ + public static class Bitmaps { + public static int encodedLength(RoaringBitmap b) { + return 4 + b.serializedSizeInBytes(); + } + + public static void encode(ByteBuf buf, RoaringBitmap b) { + // Compress the bitmap before serializing it + b.trim(); + b.runOptimize(); + ByteBuffer outBuffer = ByteBuffer.allocate(b.serializedSizeInBytes()); + try { + b.serialize(new DataOutputStream(new OutputStream() { + ByteBuffer buffer; + + OutputStream init(ByteBuffer buffer) { + this.buffer = buffer; + return this; + } + + @Override + public void close() { + } + + @Override + public void flush() { + } + + @Override + public void write(int b) { + buffer.put((byte) b); + } + + @Override + public void write(byte[] b) { + buffer.put(b); + } + + @Override + public void write(byte[] b, int off, int l) { + buffer.put(b, off, l); + } + }.init(outBuffer))); + } catch (IOException e) { + throw new RuntimeException("Exception while encoding bitmap", e); + } + byte[] bytes = outBuffer.array(); + buf.writeInt(bytes.length); + buf.writeBytes(bytes); + } + + public static RoaringBitmap decode(ByteBuf buf) { + int length = buf.readInt(); + byte[] bytes = new byte[length]; + buf.readBytes(bytes); + RoaringBitmap bitmap = new RoaringBitmap(); + try { + bitmap.deserialize(new DataInputStream(new ByteArrayInputStream(bytes))); + } catch (IOException e) { + throw new RuntimeException("Exception while decoding bitmap", e); + } + return bitmap; + } + } + /** Byte arrays are encoded with their length followed by bytes. */ public static class ByteArrays { public static int encodedLength(byte[] arr) { @@ -135,4 +207,31 @@ public static long[] decode(ByteBuf buf) { return longs; } } + + /** Bitmap arrays are encoded with the number of bitmaps followed by per-Bitmap encoding. */ + public static class BitmapArrays { + public static int encodedLength(RoaringBitmap[] bitmaps) { + int totalLength = 4; + for (RoaringBitmap b : bitmaps) { + totalLength += Bitmaps.encodedLength(b); + } + return totalLength; + } + + public static void encode(ByteBuf buf, RoaringBitmap[] bitmaps) { + buf.writeInt(bitmaps.length); + for (RoaringBitmap b : bitmaps) { + Bitmaps.encode(buf, b); + } + } + + public static RoaringBitmap[] decode(ByteBuf buf) { + int numBitmaps = buf.readInt(); + RoaringBitmap[] bitmaps = new RoaringBitmap[numBitmaps]; + for (int i = 0; i < bitmaps.length; i ++) { + bitmaps[i] = Bitmaps.decode(buf); + } + return bitmaps; + } + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index f178928006902..67e4140baee89 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -32,6 +32,7 @@ import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.*; import org.apache.spark.network.protocol.*; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.TransportFrameDecoder; import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; @@ -181,6 +182,18 @@ public void onFailure(Throwable e) { private void processStreamUpload(final UploadStream req) { assert (req.body() == null); try { + // Retain the original metadata buffer, since it will be used during the invocation of + // this method. Will be released later. + req.meta.retain(); + // Make a copy of the original metadata buffer. In benchmark, we noticed that + // we cannot respond the original metadata buffer back to the client, otherwise + // in cases where multiple concurrent shuffles are present, a wrong metadata might + // be sent back to client. This is related to the eager release of the metadata buffer, + // i.e., we always release the original buffer by the time the invocation of this + // method ends, instead of by the time we respond it to the client. This is necessary, + // otherwise we start seeing memory issues very quickly in benchmarks. + // TODO check if the way metadata buffer is handled can be further improved + ByteBuffer meta = cloneBuffer(req.meta.nioByteBuffer()); RpcResponseCallback callback = new RpcResponseCallback() { @Override public void onSuccess(ByteBuffer response) { @@ -189,13 +202,17 @@ public void onSuccess(ByteBuffer response) { @Override public void onFailure(Throwable e) { - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + // Piggyback request metadata as part of the exception error String, so we can + // respond the metadata upon a failure without changing the existing protocol. + respond(new RpcFailure(req.requestId, + JavaUtils.encodeHeaderIntoErrorString(meta.duplicate(), e))); + req.meta.release(); } }; TransportFrameDecoder frameDecoder = (TransportFrameDecoder) channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); - ByteBuffer meta = req.meta.nioByteBuffer(); - StreamCallbackWithID streamHandler = rpcHandler.receiveStream(reverseClient, meta, callback); + StreamCallbackWithID streamHandler = + rpcHandler.receiveStream(reverseClient, meta.duplicate(), callback); if (streamHandler == null) { throw new NullPointerException("rpcHandler returned a null streamHandler"); } @@ -209,12 +226,17 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { public void onComplete(String streamId) throws IOException { try { streamHandler.onComplete(streamId); - callback.onSuccess(ByteBuffer.allocate(0)); + callback.onSuccess(meta.duplicate()); } catch (Exception ex) { IOException ioExc = new IOException("Failure post-processing complete stream;" + " failing this rpc and leaving channel active", ex); + // req.meta will be released once inside callback.onFailure. Retain it one more + // time to be released in the finally block. + req.meta.retain(); callback.onFailure(ioExc); streamHandler.onFailure(streamId, ioExc); + } finally { + req.meta.release(); } } @@ -238,12 +260,26 @@ public String getID() { } } catch (Exception e) { logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + try { + // It's OK to respond the original metadata buffer here, because this is still inside + // the invocation of this method. + respond(new RpcFailure(req.requestId, + JavaUtils.encodeHeaderIntoErrorString(req.meta.nioByteBuffer(), e))); + } catch (IOException ioe) { + // No exception will be thrown here. req.meta.nioByteBuffer will not throw IOException + // because it's a NettyManagedBuffer. This try-catch block is to make compiler happy. + logger.error("Error in handling failure while invoking RpcHandler#receive() on RPC id " + + req.requestId, e); + } finally { + req.meta.release(); + } // We choose to totally fail the channel, rather than trying to recover as we do in other // cases. We don't know how many bytes of the stream the client has already sent for the // stream, it's not worth trying to recover. channel.pipeline().fireExceptionCaught(e); } finally { + // Make sure we always release the original metadata buffer by the time we exit the + // invocation of this method. Otherwise, we see memory issues fairly quickly in benchmarks. req.meta.release(); } } @@ -258,6 +294,16 @@ private void processOneWayMessage(OneWayMessage req) { } } + /** + * Make a full copy of a nio ByteBuffer. + */ + private ByteBuffer cloneBuffer(ByteBuffer buf) { + ByteBuffer clone = ByteBuffer.allocate(buf.capacity()); + clone.put(buf.duplicate()); + clone.flip(); + return clone; + } + /** * Responds to a single message with some Encodable object. If a failure occurs while sending, * it will be logged and the channel closed. diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java index b5497087634ce..956e937578bea 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -27,6 +27,7 @@ import java.util.regex.Pattern; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.apache.commons.lang3.SystemUtils; @@ -80,6 +81,17 @@ public static String bytesToString(ByteBuffer b) { return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); } + /** + * Encode the header ByteBuffer into the error string to be returned via RpcFailure. + * Use ISO_8859_1 encoding instead of UTF_8. UTF_8 will change the byte content + * for bytes larger than 127. This would render incorrect result when encoding + * decoding the index inside the PushBlockStream message. + */ + public static String encodeHeaderIntoErrorString(ByteBuffer headerBuffer, Throwable e) { + String encodedHeader = StandardCharsets.ISO_8859_1.decode(headerBuffer).toString(); + return encodedHeader + Throwables.getStackTraceAsString(e); + } + /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 646e4278811f4..8d8f564eacb99 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -363,4 +363,16 @@ public boolean useOldFetchProtocol() { return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false); } + /** + * The minimum size of a chunk when dividing a merged shuffle file in push based shuffle into + * multiple chunks. This is an optimization so that when push based shuffle merges multiple + * shuffle blocks belonging to the same shuffle partition into a merged shuffle file, it + * divides the merged shuffle file into multiple MB-sized chunks to optimize reading it later. + * A corresponding index file for each merged shuffle file will be generated indicating chunk + * boundaries. + */ + public int minChunkSizeInMergedShuffleFile() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4m"))); + } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 916c140621671..7288060bd115b 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -19,6 +19,7 @@ import java.io.*; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; @@ -168,6 +169,7 @@ public static void tearDown() { static class RpcResult { public Set successMessages; public Set errorMessages; + public Set successResponses; } private RpcResult sendRPC(String ... commands) throws Exception { @@ -210,6 +212,7 @@ private RpcResult sendRpcWithStream(String... streams) throws Exception { RpcResult res = new RpcResult(); res.successMessages = Collections.synchronizedSet(new HashSet<>()); res.errorMessages = Collections.synchronizedSet(new HashSet<>()); + res.successResponses = Collections.synchronizedSet(new HashSet<>()); for (String stream : streams) { int idx = stream.lastIndexOf('/'); @@ -247,6 +250,7 @@ private static class RpcStreamCallback implements RpcResponseCallback { @Override public void onSuccess(ByteBuffer message) { res.successMessages.add(streamId); + res.successResponses.add(JavaUtils.bytesToString(message)); sem.release(); } @@ -326,6 +330,7 @@ public void sendRpcWithStreamOneAtATime() throws Exception { RpcResult res = sendRpcWithStream(stream); assertTrue("there were error messages!" + res.errorMessages, res.errorMessages.isEmpty()); assertEquals(Sets.newHashSet(stream), res.successMessages); + assertEquals(Sets.newHashSet(stream), res.successResponses); } } @@ -336,7 +341,9 @@ public void sendRpcWithStreamConcurrently() throws Exception { streams[i] = StreamTestHelper.STREAMS[i % StreamTestHelper.STREAMS.length]; } RpcResult res = sendRpcWithStream(streams); - assertEquals(Sets.newHashSet(StreamTestHelper.STREAMS), res.successMessages); + Set streamSet = Sets.newHashSet(StreamTestHelper.STREAMS); + assertEquals(streamSet, res.successMessages); + assertEquals(streamSet, res.successResponses); assertTrue(res.errorMessages.isEmpty()); } @@ -344,22 +351,44 @@ public void sendRpcWithStreamConcurrently() throws Exception { public void sendRpcWithStreamFailures() throws Exception { // when there is a failure reading stream data, we don't try to keep the channel usable, // just send back a decent error msg. + String failStream = "fail/exception-ondata/smallBuffer"; RpcResult exceptionInCallbackResult = - sendRpcWithStream("fail/exception-ondata/smallBuffer", "smallBuffer"); + sendRpcWithStream(failStream, "smallBuffer"); assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); + assertDecodedErrorsContain(exceptionInCallbackResult.errorMessages, failStream); + failStream = "fail/null/smallBuffer"; RpcResult nullStreamHandler = - sendRpcWithStream("fail/null/smallBuffer", "smallBuffer"); + sendRpcWithStream(failStream, "smallBuffer"); assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); + assertDecodedErrorsContain(nullStreamHandler.errorMessages, failStream); // OTOH, if there is a failure during onComplete, the channel should still be fine + failStream = "fail/exception-oncomplete/smallBuffer"; RpcResult exceptionInOnComplete = - sendRpcWithStream("fail/exception-oncomplete/smallBuffer", "smallBuffer"); + sendRpcWithStream(failStream, "smallBuffer"); assertErrorsContain(exceptionInOnComplete.errorMessages, Sets.newHashSet("Failure post-processing")); + assertDecodedErrorsContain(exceptionInOnComplete.errorMessages, failStream); assertEquals(Sets.newHashSet("smallBuffer"), exceptionInOnComplete.successMessages); } + private void assertDecodedErrorsContain(Set errors, String contain) { + Set decodedErrors = Sets.newHashSet(); + for (String error : errors) { + ByteBuffer rawBuffer = ByteBuffer.wrap(error.getBytes(StandardCharsets.ISO_8859_1)); + decodedErrors.add(JavaUtils.bytesToString(rawBuffer)); + } + boolean foundMatch = false; + for (String error : decodedErrors) { + if (error.contains(contain)) { + foundMatch = true; + break; + } + } + assertTrue("Could not find decoded error containing " + contain, foundMatch); + } + private void assertErrorsContain(Set errors, Set contains) { assertEquals("Expected " + contains.size() + " errors, got " + errors.size() + "errors: " + errors, contains.size(), errors.size()); diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 00f1defbb0093..a4a1ff92ef9a0 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -57,6 +57,10 @@ com.google.guava guava + + org.roaringbitmap + RoaringBitmap + @@ -93,6 +97,11 @@ mockito-core test + + commons-io + commons-io + test + diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java new file mode 100644 index 0000000000000..567840da7e552 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java @@ -0,0 +1,69 @@ +/* + * 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; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + +/** + * A special exception type that would decode the encoded {@link PushBlockStream} from the + * exception String. This complements the encoding logic in + * {@link org.apache.spark.network.server.TransportRequestHandler}. + */ +public class BlockPushException extends RuntimeException { + private PushBlockStream header; + + /** + * String constant used for generating exception messages indicating a block to be merged + * arrives too late on the server side, and also for later checking such exceptions on the + * client side. + */ + public static final String TOO_LATE_MESSAGE_SUFFIX = + "received after merged shuffle is finalized"; + + private BlockPushException(PushBlockStream header, String message) { + super(message); + this.header = header; + } + + public static BlockPushException decodeException(String message) { + // Use ISO_8859_1 encoding instead of UTF_8. UTF_8 will change the byte content + // for bytes larger than 127. This would render incorrect result when encoding + // decoding the index inside the PushBlockStream message. + ByteBuffer rawBuffer = ByteBuffer.wrap(message.getBytes(StandardCharsets.ISO_8859_1)); + try { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(rawBuffer); + if (msgObj instanceof PushBlockStream) { + PushBlockStream header = (PushBlockStream) msgObj; + return new BlockPushException(header, StandardCharsets.UTF_8.decode(rawBuffer).toString()); + } else { + throw new UnsupportedOperationException(String.format("Cannot decode the header. " + + "Expected PushBlockStream but got %s instead", msgObj.getClass().getSimpleName())); + } + } catch (Exception e) { + return new BlockPushException(null, message); + } + } + + public PushBlockStream getHeader() { + return header; + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index e762bd2071632..d2d7fbf8b3145 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; @@ -135,4 +136,39 @@ public void onFailure(Throwable t) { hostLocalDirsCompletable.completeExceptionally(e); } } + + /** + * Push a sequence of shuffle blocks to a remote node asynchronously to be merged into + * a larger shuffle block. + * + * @param host the host of the remote node. + * @param port the port of the remote node. + * @param blockIds block ids to be pushed + * @param buffers buffers to be pushed + * @param listener the listener to receive block push status. + */ + public void pushBlocks( + String host, + int port, + String[] blockIds, + ManagedBuffer[] buffers, + BlockFetchingListener listener) { + throw new UnsupportedOperationException(); + } + + /** + * Get the meta information of the merged blocks from the remote shuffle service. + * + * @param host the host of the remote node. + * @param port the port of the remote node. + * @param blocks block ids of the merged shuffle blocks. + * @param listener the listener to receive chunk counts. + */ + public void getMergedBlocksMeta( + String host, + int port, + String[] blocks, + MergedBlocksMetaListener listener) { + throw new UnsupportedOperationException(); + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 33865a21ea914..0aee4b3d86dd7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -32,6 +32,7 @@ import com.codahale.metrics.Timer; import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; +import org.apache.spark.network.client.StreamCallbackWithID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,16 +57,27 @@ */ public class ExternalBlockHandler extends RpcHandler { private static final Logger logger = LoggerFactory.getLogger(ExternalBlockHandler.class); + private static final String SHUFFLE_BLOCK_PREFIX = "shuffle"; + private static final String SHUFFLE_CHUNK_PREFIX = "shuffleChunk"; @VisibleForTesting final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; private final ShuffleMetrics metrics; + private final MergedShuffleFileManager mergeManager; public ExternalBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { this(new OneForOneStreamManager(), - new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), new NoOpMergedShuffleFileManager()); + } + + public ExternalBlockHandler( + TransportConf conf, + File registeredExecutorFile, + MergedShuffleFileManager mergeManager) throws IOException { + this(new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), mergeManager); } @VisibleForTesting @@ -78,9 +90,19 @@ public ExternalShuffleBlockResolver getBlockResolver() { public ExternalBlockHandler( OneForOneStreamManager streamManager, ExternalShuffleBlockResolver blockManager) { + this(streamManager, blockManager, new NoOpMergedShuffleFileManager()); + } + + /** Enables mocking out the StreamManager and BlockManager. */ + @VisibleForTesting + public ExternalBlockHandler( + OneForOneStreamManager streamManager, + ExternalShuffleBlockResolver blockManager, + MergedShuffleFileManager mergeManager) { this.metrics = new ShuffleMetrics(); this.streamManager = streamManager; this.blockManager = blockManager; + this.mergeManager = mergeManager; } @Override @@ -89,28 +111,41 @@ public void receive(TransportClient client, ByteBuffer message, RpcResponseCallb handleMessage(msgObj, client, callback); } + @Override + public StreamCallbackWithID receiveStream( + TransportClient client, + ByteBuffer messageHeader, + RpcResponseCallback callback) { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(messageHeader); + if (msgObj instanceof PushBlockStream) { + PushBlockStream message = (PushBlockStream) msgObj; + checkAuth(client, message.appId); + return mergeManager.receiveBlockDataAsStream(message); + } else { + throw new UnsupportedOperationException("Unexpected message with #receiveStream: " + msgObj); + } + } + protected void handleMessage( BlockTransferMessage msgObj, TransportClient client, RpcResponseCallback callback) { - if (msgObj instanceof FetchShuffleBlocks || msgObj instanceof OpenBlocks) { + if (msgObj instanceof AbstractFetchShuffleBlocks || msgObj instanceof OpenBlocks) { final Timer.Context responseDelayContext = metrics.openBlockRequestLatencyMillis.time(); try { int numBlockIds; long streamId; - if (msgObj instanceof FetchShuffleBlocks) { - FetchShuffleBlocks msg = (FetchShuffleBlocks) msgObj; + if (msgObj instanceof AbstractFetchShuffleBlocks) { + AbstractFetchShuffleBlocks msg = (AbstractFetchShuffleBlocks) msgObj; checkAuth(client, msg.appId); - numBlockIds = 0; - if (msg.batchFetchEnabled) { - numBlockIds = msg.mapIds.length; + numBlockIds = ((AbstractFetchShuffleBlocks) msgObj).getNumBlocks(); + Iterator iterator; + if (msgObj instanceof FetchShuffleBlocks) { + iterator = new ShuffleManagedBufferIterator((FetchShuffleBlocks)msgObj); } else { - for (int[] ids: msg.reduceIds) { - numBlockIds += ids.length; - } + iterator = new ShuffleChunkManagedBufferIterator((FetchShuffleBlockChunks)msgObj); } - streamId = streamManager.registerStream(client.getClientId(), - new ShuffleManagedBufferIterator(msg), client.getChannel()); + streamId = streamManager.registerStream(client.getClientId(), iterator, client.getChannel()); } else { // For the compatibility with the old version, still keep the support for OpenBlocks. OpenBlocks msg = (OpenBlocks) msgObj; @@ -131,7 +166,29 @@ protected void handleMessage( } finally { responseDelayContext.stop(); } - + } else if (msgObj instanceof FetchMergedBlocksMeta) { + final Timer.Context responseDelayContext = + metrics.fetchMergedBlocksMetaLatencyMillis.time(); + try { + FetchMergedBlocksMeta mergedMetaReq = (FetchMergedBlocksMeta) msgObj; + checkAuth(client, mergedMetaReq.appId); + int[] chunkCounts = new int[mergedMetaReq.blockIds.length]; + for (int i = 0; i < mergedMetaReq.blockIds.length; i++) { + String[] blockIdParts = mergedMetaReq.blockIds[i].split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) { + throw new IllegalArgumentException( + "Unexpected shuffle block id format: " + mergedMetaReq.blockIds[i]); + } + chunkCounts[i] = + mergeManager.getChunkCount(mergedMetaReq.appId, Integer.parseInt(blockIdParts[1]), + Integer.parseInt(blockIdParts[3])); + logger.debug( + "Merged block {} with {} chunks", mergedMetaReq.blockIds[i], chunkCounts[i]); + } + callback.onSuccess(new MergedBlocksMeta(chunkCounts).toByteBuffer()); + } finally { + responseDelayContext.stop(); + } } else if (msgObj instanceof RegisterExecutor) { final Timer.Context responseDelayContext = metrics.registerExecutorRequestLatencyMillis.time(); @@ -156,6 +213,20 @@ protected void handleMessage( Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); + } else if (msgObj instanceof FinalizeShuffleMerge) { + final Timer.Context responseDelayContext = + metrics.finalizeShuffleMergeLatencyMillis.time(); + FinalizeShuffleMerge msg = (FinalizeShuffleMerge) msgObj; + try { + checkAuth(client, msg.appId); + MergeStatuses statuses = mergeManager.finalizeShuffleMerge(msg); + callback.onSuccess(statuses.toByteBuffer()); + } catch(IOException e) { + throw new RuntimeException(String.format("Error while finalizing shuffle merge " + + "for application %s shuffle %d", msg.appId, msg.shuffleId)); + } finally { + responseDelayContext.stop(); + } } else { throw new UnsupportedOperationException("Unexpected message: " + msgObj); } @@ -225,6 +296,10 @@ public class ShuffleMetrics implements MetricSet { private final Timer openBlockRequestLatencyMillis = new Timer(); // Time latency for executor registration latency in ms private final Timer registerExecutorRequestLatencyMillis = new Timer(); + // Time latency for processing fetch merged blocks meta request latency in ms + private final Timer fetchMergedBlocksMetaLatencyMillis = new Timer(); + // Time latency for processing finalize shuffle merge request latency in ms + private final Timer finalizeShuffleMergeLatencyMillis = new Timer(); // Block transfer rate in byte per second private final Meter blockTransferRateBytes = new Meter(); // Number of active connections to the shuffle service @@ -236,6 +311,8 @@ public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); + allMetrics.put("fetchMergedBlocksMetaLatencyMillis", fetchMergedBlocksMetaLatencyMillis); + allMetrics.put("finalizeShuffleMergeLatencyMillis", finalizeShuffleMergeLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", (Gauge) () -> blockManager.getRegisteredExecutorsSize()); @@ -254,19 +331,36 @@ private class ManagedBufferIterator implements Iterator { private int index = 0; private final Function blockDataForIndexFn; private final int size; + private final boolean requestForMergedBlockChunks; ManagedBufferIterator(OpenBlocks msg) { String appId = msg.appId; String execId = msg.execId; String[] blockIds = msg.blockIds; String[] blockId0Parts = blockIds[0].split("_"); - if (blockId0Parts.length == 4 && blockId0Parts[0].equals("shuffle")) { + if (blockId0Parts.length == 4 + && (blockId0Parts[0].equals(SHUFFLE_BLOCK_PREFIX) + || blockId0Parts[0].equals(SHUFFLE_CHUNK_PREFIX))) { final int shuffleId = Integer.parseInt(blockId0Parts[1]); + requestForMergedBlockChunks = blockId0Parts[0].equals(SHUFFLE_CHUNK_PREFIX); final int[] mapIdAndReduceIds = shuffleMapIdAndReduceIds(blockIds, shuffleId); size = mapIdAndReduceIds.length; - blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + blockDataForIndexFn = + index -> { + if (requestForMergedBlockChunks) { + return mergeManager.getMergedBlockData( + msg.appId, shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + } else { + return blockManager.getBlockData( + msg.appId, + msg.execId, + shuffleId, + mapIdAndReduceIds[index], + mapIdAndReduceIds[index + 1]); + } + }; } else if (blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { + requestForMergedBlockChunks = false; final int[] rddAndSplitIds = rddAndSplitIds(blockIds); size = rddAndSplitIds.length; blockDataForIndexFn = index -> blockManager.getRddBlockData(appId, execId, @@ -293,14 +387,18 @@ 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")) { + if (blockIdParts.length != 4 || + (!requestForMergedBlockChunks && !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) || + (requestForMergedBlockChunks && !blockIdParts[0].equals(SHUFFLE_CHUNK_PREFIX))) { throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[i]); } if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockIds[i]); } + // For regular blocks this is mapId. For chunks this is reduceId. mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); + // For regular blocks this is reduceId. For chunks this is chunkId. mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } return mapIdAndReduceIds; @@ -373,6 +471,87 @@ public ManagedBuffer next() { } } + private class ShuffleChunkManagedBufferIterator implements Iterator { + + private int reduceIdx = 0; + private int chunkIdx = 0; + + private final String appId; + private final String execId; + private final int shuffleId; + private final int[] reduceIds; + private final int[][] chunkIds; + + ShuffleChunkManagedBufferIterator(FetchShuffleBlockChunks msg) { + appId = msg.appId; + execId = msg.execId; + shuffleId = msg.shuffleId; + reduceIds = msg.reduceIds; + chunkIds = msg.chunkIds; + } + + @Override + public boolean hasNext() { + // reduceIds.length must equal to chunkIds.length, and the passed in FetchShuffleBlockChunks + // must have non-empty reduceIds and chunkIds, see the checking logic in + // OneForOneBlockFetcher. + assert(reduceIds.length != 0 && reduceIds.length == chunkIds.length); + return reduceIdx < reduceIds.length && chunkIdx < chunkIds[reduceIdx].length; + } + + @Override + public ManagedBuffer next() { + ManagedBuffer block = mergeManager.getMergedBlockData( + appId, shuffleId, reduceIds[reduceIdx], chunkIds[reduceIdx][chunkIdx]); + if (chunkIdx < chunkIds[reduceIdx].length - 1) { + chunkIdx += 1; + } else { + chunkIdx = 0; + reduceIdx += 1; + } + metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); + return block; + } + } + + /** + * Dummy implementation of merged shuffle file manager. Suitable for when this feature + * is not turned on. + */ + private static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager { + + @Override + public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public void registerApplication(String appId, String relativeAppPath) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public ManagedBuffer getMergedBlockData( + String appId, int shuffleId, int reduceId, int chunkId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + + @Override + public int getChunkCount(String appId, int shuffleId, int reduceId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } + } + @Override public void channelActive(TransportClient client) { metrics.activeConnections.inc(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index 76e23e7c69d2d..ab789d1ad7d35 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -32,6 +33,7 @@ import org.apache.spark.network.shuffle.protocol.*; import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; @@ -116,6 +118,113 @@ public void fetchBlocks( } } + @Override + public void pushBlocks( + String host, + int port, + String[] blockIds, + ManagedBuffer[] buffers, + BlockFetchingListener listener) { + checkInit(); + assert blockIds.length == buffers.length : "Number of block ids and buffers do not match."; + + Map buffersWithId = new HashMap<>(); + for (int i = 0; i < blockIds.length; i++) { + buffersWithId.put(blockIds[i], buffers[i]); + } + logger.debug("Push shuffle blocks to {}:{} with {} blocks", host, port, blockIds.length); + try { + RetryingBlockFetcher.BlockFetchStarter blockPushStarter = + (blockIds1, listener1) -> { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockPusher(client, appId, blockIds1, listener1, buffersWithId).start(); + }; + int maxRetries = conf.maxIORetries(); + if (maxRetries > 0) { + new RetryingBlockFetcher(conf, blockPushStarter, blockIds, listener).start(); + } else { + blockPushStarter.createAndStart(blockIds, listener); + } + } catch (Exception e) { + logger.error("Exception while beginning pushBlocks", e); + for (String blockId : blockIds) { + listener.onBlockFetchFailure(blockId, e); + } + } + } + + @Override + public void getMergedBlocksMeta( + String host, + int port, + String[] blocks, + MergedBlocksMetaListener listener) { + checkInit(); + logger.debug("Get merged blocks meta from {}:{} with {} blocks", host, port, blocks.length); + try { + TransportClient client = clientFactory.createClient(host, port); + FetchMergedBlocksMeta mergedBlocksMetaReq = new FetchMergedBlocksMeta(appId, blocks); + client.sendRpc(mergedBlocksMetaReq.toByteBuffer(), new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + + MergedBlocksMeta blocksMeta = + (MergedBlocksMeta) BlockTransferMessage.Decoder.fromByteBuffer(response); + logger.trace("Successfully got merged blocks meta {}", blocksMeta); + listener.onSuccess(blocks, blocksMeta.numChunks); + } + + @Override + public void onFailure(Throwable e) { + logger.error("Failed while getting merged blocks meta", e); + listener.onFailure(blocks, e); + } + }); + } catch (Exception e) { + logger.error("Exception while getting merged blocks meta", e); + listener.onFailure(blocks, e); + } + } + + /** + * Invoked by Spark driver to notify external shuffle services to finalize the shuffle merge + * for a given shuffle. This allows the driver to start the shuffle reducer stage after properly + * finishing the shuffle merge process associated with the shuffle mapper stage. + * + * @param host host of shuffle server + * @param port port of shuffle server. + * @param shuffleId shuffle ID of the shuffle to be finalized + * @param listener the listener to receive MergeStatuses + */ + // TODO Might be better create a separate shuffle client similar to MesosExternalShuffleClient, + // TODO as this is going to be used by the driver, to avoid having to initialize an + // TODO ExternalShuffleClient. + public void finalizeShuffleMerge( + String host, + int port, + int shuffleId, + MergeFinalizerListener listener) { + checkInit(); + try { + TransportClient client = clientFactory.createClient(host, port); + ByteBuffer finalizeShuffleMerge = new FinalizeShuffleMerge(appId, shuffleId).toByteBuffer(); + client.sendRpc(finalizeShuffleMerge, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + listener.onShuffleMergeSuccess( + (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response)); + } + + @Override + public void onFailure(Throwable e) { + listener.onShuffleMergeFailure(e); + } + }); + } catch (Exception e) { + logger.error("Exception while sending finalizeShuffleMerge request", e); + } + } + @Override public MetricSet shuffleMetrics() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java new file mode 100644 index 0000000000000..2d353bf2c5350 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java @@ -0,0 +1,35 @@ +/* + * 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; + +import java.util.EventListener; + +import org.apache.spark.network.shuffle.protocol.MergeStatuses; + +public interface MergeFinalizerListener extends EventListener { + /** + * Called once upon successful response on finalize shuffle merge on a remote shuffle service. + * The returned {@link MergeStatuses} is passed to the listener for further processing + */ + void onShuffleMergeSuccess(MergeStatuses statuses); + + /** + * Called once upon failure + */ + void onShuffleMergeFailure(Throwable e); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java new file mode 100644 index 0000000000000..3ca66e06aad51 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java @@ -0,0 +1,41 @@ +/* + * 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; + +import java.util.EventListener; + +/** + * Listener for receiving success or failure events when fetching meta of merged blocks. + */ +public interface MergedBlocksMetaListener extends EventListener { + + /** + * Called after successfully receiving the meta of merged blocks. Currently, the meta only + * includes the count of chunks in a merged block. + * @param blocks merged block Ids. + * @param numChunks number of chunks per merged block. + */ + void onSuccess(String[] blocks, int[] numChunks); + + /** + * Called when there is an exception while fetching the meta of merged blocks. + * @param blocks merged block Ids. + * @param exception exception getting chunk counts. + */ + void onFailure(String[] blocks, Throwable exception); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java new file mode 100644 index 0000000000000..14044e49b454b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -0,0 +1,103 @@ +/* + * 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; + +import java.io.IOException; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.StreamCallbackWithID; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.MergeStatuses; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + + +/** + * The MergedShuffleFileManager is used to process push based shuffle when enabled. It works + * along side {@link ExternalShuffleBlockHandler} and serves as an RPCHandler for + * {@link org.apache.spark.network.server.RpcHandler#receiveStream}, where it processes the + * remotely pushed streams of shuffle blocks to merge them into merged shuffle files. Right + * now, push based shuffle can only be enabled when external shuffle service in YARN mode + * is used. + */ +public interface MergedShuffleFileManager { + /** + * Provides the stream callback used to process a remotely pushed block. The callback is + * used by the {@link org.apache.spark.network.client.StreamInterceptor} installed on the + * channel to process the block data in the channel outside of the message frame. + * + * @param msg metadata of the remotely pushed blocks. This is processed inside the message frame + * @return A stream callback to process the block data in streaming fashion as it arrives + */ + StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg); + + /** + * Handles the request to finalize shuffle merge for a given shuffle. + * + * @param msg contains appId and shuffleId to uniquely identify a shuffle to be finalized + * @return The statuses of the merged shuffle partitions for the given shuffle on this + * shuffle service + * @throws IOException + */ + MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException; + + /** + * Registers an application when it starts. This provides the application specific path + * so MergedShuffleFileManager knows where to store and look for shuffle data for a + * given application. Right now, this is invoked by YarnShuffleService. + * + * @param appId application ID + * @param relativeAppPath The relative path which is application unique. The actual directory + * path is split into 2 parts, where the first half is one of the + * several configured local dirs that're shared across all applications + * and the second half is application unique. + */ + void registerApplication(String appId, String relativeAppPath); + + /** + * Invoked when an application finishes. This cleans up any remaining metadata associated with + * this application, and optionally deletes the application specific directory path. + * + * @param appId application ID + * @param cleanupLocalDirs flag indicating whether MergedShuffleFileManager should handle + * deletion of local dirs itself. Ideally, we should be able to delegate + * to YARN to handle local dir deletion in YARN mode. This does not work + * as expected yet. See LIHADOOP-52202. + */ + void applicationRemoved(String appId, boolean cleanupLocalDirs); + + /** + * Get the buffer for a given merged shuffle chunk when serving merged shuffle to reducers + * + * @param appId application ID + * @param shuffleId shuffle ID + * @param reduceId reducer ID + * @param chunkId merged shuffle file chunk ID + * @return The {@link ManagedBuffer} for the given merged shuffle chunk + */ + ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId); + + /** + * Get the number of chunks for a given merged shuffle file. + * + * @param appId application ID + * @param shuffleId shuffle ID + * @param reduceId reducer ID + * @return number of chunks + */ + int getChunkCount(String appId, int shuffleId, int reduceId); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index ec2e3dce661d9..07b81aa3f9545 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -25,6 +25,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,7 @@ import org.apache.spark.network.client.StreamCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.shuffle.protocol.AbstractFetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; @@ -52,6 +54,9 @@ public class OneForOneBlockFetcher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); + private static final String SHUFFLE_BLOCK_PREFIX = "shuffle"; + private static final String SHUFFLE_CHUNK_PREFIX = "shuffleChunk"; + private final TransportClient client; private final BlockTransferMessage message; private final String[] blockIds; @@ -89,22 +94,68 @@ public OneForOneBlockFetcher( if (blockIds.length == 0) { throw new IllegalArgumentException("Zero-sized blockIds array"); } - if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { - this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); + if (!transportConf.useOldFetchProtocol() && areShuffleBlocksOrChunks(blockIds)) { + this.message = createFetchShuffleBlocksOrChunksMsg(appId, execId, blockIds); } else { this.message = new OpenBlocks(appId, execId, blockIds); } } - private boolean isShuffleBlocks(String[] blockIds) { + /** + * Check if the array of block IDs are all shuffle block IDs. With push based shuffle, + * the shuffle block ID could be either unmerged shuffle block IDs or merged shuffle chunk + * IDs. For a given stream of shuffle blocks to be fetched in one request, they would be either + * all unmerged shuffle blocks or all merged shuffle chunks. + * @param blockIds block ID array + * @return whether the array contains only shuffle block IDs + */ + private boolean areShuffleBlocksOrChunks(String[] blockIds) { for (String blockId : blockIds) { - if (!blockId.startsWith("shuffle_")) { + if (!blockId.startsWith(SHUFFLE_BLOCK_PREFIX) && + !blockId.startsWith(SHUFFLE_CHUNK_PREFIX)) { return false; } } return true; } + /** Creates either a {@link FetchShuffleBlocks} or {@link FetchShuffleBlockChunks} message. */ + private AbstractFetchShuffleBlocks createFetchShuffleBlocksOrChunksMsg( + String appId, String execId, String[] blockIds) { + if (blockIds[0].startsWith(SHUFFLE_CHUNK_PREFIX)) { + return createFetchShuffleBlockChunksMsg(appId, execId, blockIds); + } else { + return createFetchShuffleBlocksMsg(appId, execId, blockIds); + } + } + + /** + * Creates FetchShuffleBlockChunks message. + */ + private FetchShuffleBlockChunks createFetchShuffleBlockChunksMsg( + String appId, String execId, String[] blockIds) { + String[] firstBlock = splitBlockId(blockIds[0]); + int shuffleId = Integer.parseInt(firstBlock[1]); + HashMap> reduceIdsToChunkIds = new HashMap<>(); + for (String blockId : blockIds) { + String[] blockIdParts = splitBlockId(blockId); + if (Integer.parseInt(blockIdParts[1]) != shuffleId) { + throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); + } + int reduceId = Integer.parseInt(blockIdParts[2]); + if (!reduceIdsToChunkIds.containsKey(reduceId)) { + reduceIdsToChunkIds.put(reduceId, new ArrayList<>()); + } + reduceIdsToChunkIds.get(reduceId).add(Integer.parseInt(blockIdParts[3])); + } + int[] reduceIds = Ints.toArray(reduceIdsToChunkIds.keySet()); + int[][] chunkIdArr = new int[reduceIds.length][]; + for (int i = 0; i < chunkIdArr.length; i++) { + chunkIdArr[i] = Ints.toArray(reduceIdsToChunkIds.get(reduceIds[i])); + } + return new FetchShuffleBlockChunks(appId, execId, shuffleId, reduceIds, chunkIdArr); + } + /** * Analyze the pass in blockIds and create FetchShuffleBlocks message. * The blockIds has been sorted by mapId and reduceId. It's produced in @@ -150,7 +201,8 @@ private String[] splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); // For batch block id, the format contains shuffleId, mapId, begin reduceId, end reduceId. // For single block id, the format contains shuffleId, mapId, educeId. - if (blockIdParts.length < 4 || blockIdParts.length > 5 || !blockIdParts[0].equals("shuffle")) { + if (blockIdParts.length < 4 || blockIdParts.length > 5 + || !(blockIdParts[0].equals("shuffle") || blockIdParts[0].equals(SHUFFLE_CHUNK_PREFIX))) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + blockId); } @@ -167,9 +219,16 @@ public void onSuccess(int chunkIndex, ManagedBuffer buffer) { @Override public void onFailure(int chunkIndex, Throwable e) { - // On receipt of a failure, fail every block from chunkIndex onwards. - String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); - failRemainingBlocks(remainingBlockIds, e); + // If failed block is a merged block, we only fail this block and do not + // fail the remaining blocks. The failed merged block will be retried by + // falling back to fetching the original unmerged blocks. + if (isMergedBlock(blockIds[chunkIndex])) { + failSingleMergedBlock(blockIds[chunkIndex], e); + } else { + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, e); + } } } @@ -221,6 +280,23 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { } } + private void failSingleMergedBlock(String mergedBlockId, Throwable e) { + try { + listener.onBlockFetchFailure(mergedBlockId, e); + } catch (Exception e2) { + logger.error("Error in block fetch failure callback", e2); + } + } + + /** Verify if a given block id represents a merged block. */ + // TODO this needs to be updated to support merged shuffle chunks. Right now + // TODO fallback handling for merged shuffle chunk is not enabled. + private boolean isMergedBlock(String blockId) { + String[] blockIdParts = blockId.split("_"); + return blockIdParts.length == 4 && blockIdParts[0].equals("shuffle") + && blockIdParts[2].equals("-1"); + } + private class DownloadCallback implements StreamCallback { private DownloadFileWritableChannel channel = null; @@ -251,9 +327,14 @@ public void onComplete(String streamId) throws IOException { @Override public void onFailure(String streamId, Throwable cause) throws IOException { channel.close(); - // On receipt of a failure, fail every block from chunkIndex onwards. - String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); - failRemainingBlocks(remainingBlockIds, cause); + // Do not fail the remaining blocks if the failed block is a merged block. + if (isMergedBlock(blockIds[chunkIndex])) { + failSingleMergedBlock(blockIds[chunkIndex], cause); + } else { + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, cause); + } targetFile.delete(); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java new file mode 100644 index 0000000000000..feee7ba5c0279 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -0,0 +1,125 @@ +/* + * 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; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; + +/** + * Similar to {@link OneForOneBlockFetcher}, but for pushing blocks to remote shuffle service to + * be merged instead of for fetching them from remote shuffle services. This is used by + * ShuffleWriter when the block push process is initiated. The supplied BlockFetchingListener + * is used to handle the success or failure in pushing each blocks. + */ +public class OneForOneBlockPusher { + private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockPusher.class); + + private final TransportClient client; + private final String appId; + private final String[] blockIds; + private final BlockFetchingListener listener; + private final RpcResponseCallback callback; + private final Map buffers; + + public OneForOneBlockPusher( + TransportClient client, + String appId, + String[] blockIds, + BlockFetchingListener listener, + Map buffers) { + this.client = client; + this.appId = appId; + this.blockIds = blockIds; + this.listener = listener; + this.callback = new BlockPushCallback(); + this.buffers = buffers; + } + + private class BlockPushCallback implements RpcResponseCallback { + + @Override + public void onSuccess(ByteBuffer response) { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); + if (msgObj instanceof PushBlockStream) { + PushBlockStream respMsg = (PushBlockStream) msgObj; + // On receipt of a successful block push + listener.onBlockFetchSuccess(respMsg.blockId, + new NioManagedBuffer(ByteBuffer.allocate(0))); + } else { + throw new UnsupportedOperationException("Unexpected response: " + msgObj); + } + } + + @Override + public void onFailure(Throwable e) { + // Decode PushBlockStream header information from the exception message + BlockPushException decodedExp = BlockPushException.decodeException(e.getMessage()); + PushBlockStream respMsg = decodedExp.getHeader(); + if (respMsg != null) { + // If a PushBlockStream header can be decoded from the exception message, this exception + // is generated by the server, and thus is not retriable. Instead of failing all remaining + // requests, we only fail the one that's actually failed. + String[] targetBlockId = Arrays.copyOfRange(blockIds, respMsg.index, respMsg.index + 1); + failRemainingBlocks(targetBlockId, decodedExp); + } else { + // If a PushBlockStream header cannot be decoded from the exception message, this + // exception is not generated by the server when processing a PushBlockStream request. + // In this case, the exception could potentially be handled by retry. Since we do not + // have access to index in this case, we have to retry the entire batch. + failRemainingBlocks(blockIds, e); + } + } + } + + private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { + for (String blockId : failedBlockIds) { + try { + listener.onBlockFetchFailure(blockId, e); + } catch (Exception e2) { + logger.error("Error in block push failure callback", e2); + } + } + } + + /** + * Begins the block pushing process, calling the listener with every block pushed. + */ + public void start() { + logger.debug("Start pushing {} blocks", blockIds.length); + for (int i = 0; i < blockIds.length; i++) { + if (!buffers.containsKey(blockIds[i])) { + logger.warn("Ignore block push request for block id {} since no buffer " + + "is found for this block.", blockIds[i]); + } else { + ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); + client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), callback); + } + } + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java new file mode 100644 index 0000000000000..3b7d0d6148d61 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -0,0 +1,685 @@ +/* + * 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; + +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.Weigher; +import com.google.common.collect.Maps; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.StreamCallbackWithID; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.MergeStatuses; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * An implementation of MergedShuffleFileManager that provides the most essential shuffle + * service processing logic to support push based shuffle. + */ +public class RemoteBlockPushResolver implements MergedShuffleFileManager { + + private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); + + private final Path[] localDirs; + private final ConcurrentMap appsRelativePath; + private final ConcurrentMap partitions; + + private final Executor directoryCleaner; + private final TransportConf conf; + private final int minChunkSize; + + private final LoadingCache indexCache; + + @SuppressWarnings("UnstableApiUsage") + public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { + this.conf = conf; + this.localDirs = new Path[localDirs.length]; + for (int i = 0; i < localDirs.length; i++) { + this.localDirs[i] = Paths.get(localDirs[i]); + } + this.partitions = Maps.newConcurrentMap(); + this.appsRelativePath = Maps.newConcurrentMap(); + this.directoryCleaner = Executors.newSingleThreadExecutor( + // Add `spark` prefix because it will run in NM in Yarn mode. + NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); + this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); + String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); + CacheLoader indexCacheLoader = + new CacheLoader() { + public ShuffleIndexInformation load(File file) throws IOException { + return new ShuffleIndexInformation(file); + } + }; + indexCache = CacheBuilder.newBuilder() + .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) + .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) + .build(indexCacheLoader); + } + + /** + * Given an ID that uniquely identifies a given shuffle partition of an application, retrieves + * the associated metadata. If not present and the corresponding merged shuffle does not exist, + * initializes the metadata. + */ + private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( + AppShufflePartitionId id) { + return partitions.computeIfAbsent(id, key -> { + // It only gets here when the key is not present in the map. This could either + // be the first time the merge manager receives a pushed block for a given application + // shuffle partition, or after the merged shuffle file is finalized. We handle these + // two cases accordingly by checking if the file already exists. + try { + File mergedShuffleFile = getMergedShuffleFile(key); + if (mergedShuffleFile.exists()) { + return null; + } else { + return new AppShufflePartitionInfo(mergedShuffleFile, getMergedIndexFile(id)); + } + } catch (IOException e) { + throw new RuntimeException(String.format( + "Cannot initialize merged shuffle partition %s", key.toString()), e); + } + }); + } + + @Override + public int getChunkCount(String appId, int shuffleId, int reduceId) { + AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); + File indexFile = getMergedIndexFile(id); + if (!indexFile.exists()) { + throw new RuntimeException( + String.format("Application merged shuffle index file is not found (id=%s)", + id.toString())); + } + int size = (int) indexFile.length(); + // First entry is the zero offset + return (size / Long.BYTES) - 1; + } + + @SuppressWarnings("UnstableApiUsage") + @Override + public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) { + AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); + File mergedShuffleFile = getMergedShuffleFile(id); + if (!mergedShuffleFile.exists()) { + throw new RuntimeException( + String.format("Application merged shuffle file is not found (id=%s)", id.toString())); + } + File indexFile = getMergedIndexFile(id); + try { + // If we get here, the merged shuffle file should have been properly finalized. Thus we can + // use the file length to determine the size of the merged shuffle block. + ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile); + ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId); + return new FileSegmentManagedBuffer( + conf, + mergedShuffleFile, + shuffleIndexRecord.getOffset(), + shuffleIndexRecord.getLength()); + } catch (ExecutionException e) { + throw new RuntimeException("Failed to open file: " + indexFile, e); + } + } + + /** + * The logic here is consistent with + * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile + */ + // TODO should we use subDirsPerLocalDir to potentially reduce inode size? + private File getFile(String appId, String filename) { + int hash = JavaUtils.nonNegativeHash(filename); + Path localDir = localDirs[hash % localDirs.length]; + Path relativeMergeDir = Preconditions.checkNotNull( + appsRelativePath.get(appId), "application " + appId + " is not registered."); + return new File(localDir.resolve(relativeMergeDir).toFile(), filename); + } + + private File getMergedShuffleFile(AppShufflePartitionId id) { + String fileName = id.generateFileName(); + return getFile(id.appId, fileName); + } + + private File getMergedIndexFile(AppShufflePartitionId id) { + String indexName = id.generateIndexFileName(); + return getFile(id.appId, indexName); + } + + @Override + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); + Path relativeMergeDir = appsRelativePath.remove(appId); + Iterator> iterator = + partitions.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + AppShufflePartitionId partitionId = entry.getKey(); + AppShufflePartitionInfo partition = entry.getValue(); + if (appId.equals(partitionId.appId)) { + iterator.remove(); + try { + partition.channel.close(); + } catch (IOException e) { + logger.error("Error closing merged shuffle file for {}", partitionId); + } + } + } + + if (cleanupLocalDirs) { + Path[] dirs = Arrays.stream(localDirs) + .map(dir -> dir.resolve(relativeMergeDir)).toArray(Path[]::new); + directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); + } + } + + /** + * Synchronously delete local dirs, executed in a separate thread. + */ + private void deleteExecutorDirs(Path[] dirs) { + for (Path localDir : dirs) { + try { + if (Files.exists(localDir)) { + JavaUtils.deleteRecursively(localDir.toFile()); + logger.debug("Successfully cleaned up directory: {}", localDir); + } + } catch (Exception e) { + logger.error("Failed to delete directory: " + localDir, e); + } + } + } + + @Override + public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { + // Retrieve merged shuffle file metadata + String[] blockIdParts = msg.blockId.split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { + throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId); + } + AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId, + Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3])); + int mapId = Integer.parseInt(blockIdParts[2]); + AppShufflePartitionInfo partitionInfoBeforeCheck = + getOrCreateAppShufflePartitionInfo(partitionId); + + // Here partitionInfo will be null in 2 cases: + // 1) The request is received for a block that has already been merged, this is possible due + // to the retry logic. + // 2) The request is received after the merged shuffle is finalized, thus is too late. + // + // For case 1, we will drain the data in the channel and just respond success + // to the client. This is required because the response of the previously merged + // block will be ignored by the client, per the logic in RetryingBlockFetcher. + // Note that the netty server should receive data for a given block id only from 1 channel + // at any time. The block should be pushed only from successful maps, thus there should be + // only 1 source for a given block at any time. Although the netty client might retry sending + // this block to the server multiple times, the data of the same block always arrives from the + // same channel thus the server should have already processed the previous request of this + // block before seeing it again in the channel. This guarantees that we can simply just + // check the bitmap to determine if a block is a duplicate or not. + // + // For case 2, we will also drain the data in the channel, but throw an exception in + // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way, + // the client will be notified of the failure but the channel will remain active. Keeping + // the channel alive is important because the same channel could be reused by multiple map + // tasks in the executor JVM, which belongs to different stages. While one of the shuffles + // in these stages is finalized, the others might still be active. Tearing down the channel + // on the server side will disrupt these other on-going shuffle merges. It's also important + // to notify the client of the failure, so that it can properly halt pushing the remaining + // blocks upon receiving such failures to preserve resources on the server/client side. + // + // Speculative execution would also raise a possible scenario with duplicate blocks. Although + // speculative execution would kill the slower task attempt, leading to only 1 task attempt + // succeeding in the end, there is no guarantee that only one copy of the block will be + // pushed. This is due to our handling of block push process outside of the map task, thus + // it is possible for the speculative task attempt to initiate the block push process before + // getting killed. When this happens, we need to distinguish the duplicate blocks as they + // arrive. More details on this is explained in later comments. + + // Track if the block is received after shuffle merge finalize + final boolean isTooLate = partitionInfoBeforeCheck == null; + // Check if the given block is already merged by checking the bitmap against the given mapId + final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null + && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck; + + return new StreamCallbackWithID() { + private int length = 0; + private boolean canWrite = true; + // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly + private List deferredBufs; + + @Override + public String getID() { + return msg.blockId; + } + + /** + * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of + * the block data written to file. In case of failure during writing block to file, + * we use the information tracked in partitionInfo to overwrite the corrupt block + * when writing the new block. + */ + private void writeBuf(ByteBuffer buf) throws IOException { + while (buf.hasRemaining()) { + if (partitionInfo.isEncounteredFailure()) { + length += partitionInfo.channel.write(buf, partitionInfo.getPosition() + length); + } else { + length += partitionInfo.channel.write(buf); + } + } + } + + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + // If partition info is null, ignore the requests. It could only be + // null either when a request is received after the shuffle file is + // finalized or when a request is for a duplicate block. + if (partitionInfo == null) { + return; + } + // When handling the block data using StreamInterceptor, it can help to reduce the amount + // of data that needs to be buffered in memory since it does not wait till the completion + // of the frame before handling the message, thus releasing the ByteBuf earlier. However, + // this also means it would chunk a block into multiple buffers. Here, we want to preserve + // the benefit of handling the block data using StreamInterceptor as much as possible while + // providing the guarantee that one block would be continuously written to the merged + // shuffle file before the next block starts. For each shuffle partition, we would track + // the current map id to make sure only block matching the map id can be written to disk. + // If one server thread sees the block being handled is the current block, it would + // directly write the block to disk. Otherwise, it would buffer the block chunks in memory. + // If the block becomes the current block before we see the end of it, we would then dump + // all buffered block data to disk and write the remaining portions of the block directly + // to disk as well. This way, we avoid having to buffer the entirety of every blocks in + // memory, while still providing the necessary guarantee. + synchronized (partitionInfo) { + // If the key is no longer present in the map, it means the shuffle merge has already + // been finalized. We should thus ignore the data and just drain the remaining bytes of + // this message. This check should be placed inside the synchronized block to make sure + // that checking the key is still present and processing the data is atomic. + if (!partitions.containsKey(partitionId)) { + // TODO is it necessary to dereference deferredBufs? + deferredBufs = null; + return; + } + // Check whether we can write to disk + if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { + // Check if this is a duplicate block generated by speculative tasks. With speculative + // tasks, we could receive the same block from 2 different sources at the same time. + // One of them is going to be the first to set the currentMapId. When that block does + // so, it's going to see the currentMapId initially as -1. After it sets the + // currentMapId, it's going to write some data to disk, thus increasing the length + // counter. The other duplicate block is going to see the currentMapId already set to + // its mapId. However, it hasn't written any data yet. If the first block gets written + // completely and resets the currentMapId to -1 before the processing for the second + // block finishes, we can just check the bitmap to identify the second as a duplicate. + if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || + partitionInfo.mapTracker.contains(mapId)) { + deferredBufs = null; + return; + } + if (partitionInfo.getCurrentMapId() < 0) { + partitionInfo.setCurrentMapId(mapId); + } + + // If we got here, it's safe to write the block data to the merged shuffle file. We + // first write any deferred block chunk buffered in memory, then write the remaining + // of the block. + if (deferredBufs != null && !deferredBufs.isEmpty()) { + for (ByteBuffer deferredBuf : deferredBufs) { + writeBuf(deferredBuf); + } + deferredBufs = null; + } + writeBuf(buf); + // If we got here, it means we successfully write the current chunk of block to merged + // shuffle file. If we encountered failure while writing the previous block, we should + // reset the file channel position and the status of partitionInfo to indicate that we + // have recovered from previous disk write failure. However, we do not update the + // position tracked by partitionInfo here. That is only updated while the entire block + // is successfully written to merged shuffle file. + if (partitionInfo.isEncounteredFailure()) { + partitionInfo.channel.position(partitionInfo.getPosition() + length); + partitionInfo.setEncounteredFailure(false); + } + } else { + // If we cannot write to disk, we buffer the current block chunk in memory so it could + // potentially be written to disk later. We take our best effort without guarantee + // that the block will be written to disk. If the block data is divided into multiple + // chunks during TCP transportation, each #onData invocation is an attempt to write + // the block to disk. If the block is still not written to disk after all #onData + // invocations, the final #onComplete invocation is the last attempt to write the + // block to disk. If we still couldn't write this block to disk after this, we give up + // on this block push request and respond failure to client. We could potentially + // buffer the block longer or wait for a few iterations inside #onData or #onComplete + // to increase the chance of writing the block to disk, however this would incur more + // memory footprint or decrease the server processing throughput for the shuffle + // service. In addition, during test we observed that by randomizing the order in + // which clients sends block push requests batches, only ~0.5% blocks failed to be + // written to disk due to this reason. We thus decide to optimize for server + // throughput and memory usage. + if (deferredBufs == null) { + deferredBufs = new LinkedList<>(); + } + // Write the buffer to the in-memory deferred cache + ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining()); + deferredBuf.put(buf); + deferredBuf.flip(); + deferredBufs.add(deferredBuf); + } + } + } + + @Override + public void onComplete(String streamId) throws IOException { + if (partitionInfo == null) { + if (isTooLate) { + // Throw an exception here so the block data is drained from channel and server + // responds RpcFailure to the client. + throw new RuntimeException(String.format("Block %s %s", msg.blockId, + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + } else { + // For duplicate block that is received before the shuffle merge finalizes, the + // server should respond success to the client. + return; + } + } + // TODO should the merge manager check for the merge completion ratio here and finalize + // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early + // TODO and the file channel can be closed even if finalize merge request is somehow not + // TODO received from the driver? If so, then we need to know # maps for this shuffle. + + synchronized (partitionInfo) { + // When this request initially got to the server, the shuffle merge finalize request + // was not received yet. By the time we finish reading this message, the shuffle merge + // however is already finalized. We should thus respond RpcFailure to the client. + if (!partitions.containsKey(partitionId)) { + deferredBufs = null; + throw new RuntimeException(String.format("Block %s %s", msg.blockId, + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + } + // Check if we can commit this block + if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { + // Identify duplicate block generated by speculative tasks. We respond success to + // the client in cases of duplicate even though no data is written. + if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || + partitionInfo.mapTracker.contains(mapId)) { + deferredBufs = null; + return; + } + if (partitionInfo.getCurrentMapId() < 0 && deferredBufs != null + && !deferredBufs.isEmpty()) { + for (ByteBuffer deferredBuf : deferredBufs) { + writeBuf(deferredBuf); + } + deferredBufs = null; + } + long updatedPos = partitionInfo.getPosition() + length; + if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { + partitionInfo.updateLastChunkOffset(updatedPos); + } + partitionInfo.setPosition(updatedPos); + partitionInfo.setCurrentMapId(-1); + + // update merged results + partitionInfo.blockMerged(mapId); + } else { + deferredBufs = null; + canWrite = false; + throw new RuntimeException(String.format("Couldn't find an opportunity to write " + + "block %s to merged shuffle", msg.blockId)); + } + } + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + logger.error("Encountered issue when merging shuffle partition block {}", msg, cause); + // Only update partitionInfo if the failure corresponds to a valid request. If the + // request is too late, i.e. received after shuffle merge finalize, #onFailure will + // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity + // to write the block data to disk, we should also ignore here. + if (canWrite && partitionInfo != null && partitions.containsKey(partitionId)) { + synchronized (partitionInfo) { + partitionInfo.setCurrentMapId(-1); + partitionInfo.setEncounteredFailure(true); + } + } + } + }; + } + + @Override + public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { + logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId); + List bitmaps = new LinkedList<>(); + List reduceIds = new LinkedList<>(); + List sizes = new LinkedList<>(); + Iterator> iterator = + partitions.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + AppShufflePartitionId partitionId = entry.getKey(); + AppShufflePartitionInfo partition = entry.getValue(); + if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) { + synchronized (partition) { + iterator.remove(); + // Get rid of any partial block data at the end of the file. This could either + // be due to failure or a request still being processed when the shuffle + // merge gets finalized. + try { + partition.channel.truncate(partition.getPosition()); + if (partition.getPosition() != partition.getLastChunkOffset()) { + partition.updateLastChunkOffset(partition.getPosition()); + } + bitmaps.add(partition.mapTracker); + reduceIds.add(partitionId.reduceId); + sizes.add(partition.getPosition()); + } catch (IOException ioe) { + logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId, + msg.shuffleId, partitionId.reduceId, ioe); + } finally { + try { + partition.channel.close(); + partition.indexWriteStream.close(); + } catch (IOException closeEx) { + logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId, + msg.shuffleId, partitionId.reduceId, closeEx); + } + } + } + } + } + logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId); + return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), + Ints.toArray(reduceIds), Longs.toArray(sizes)); + } + + @Override + public void registerApplication(String appId, String relativeAppPath) { + logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); + appsRelativePath.put(appId, Paths.get(relativeAppPath)); + } + + /** + * ID that uniquely identifies a shuffle partition for an application. This is used to key + * the metadata tracked for each shuffle partition that's being actively merged. + */ + public static class AppShufflePartitionId { + public final String appId; + public final int shuffleId; + public final int reduceId; + + AppShufflePartitionId(String appId, int shuffleId, int reduceId) { + this.appId = appId; + this.shuffleId = shuffleId; + this.reduceId = reduceId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AppShufflePartitionId that = (AppShufflePartitionId) o; + return shuffleId == that.shuffleId && reduceId == that.reduceId + && Objects.equal(appId, that.appId); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, shuffleId, reduceId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("shuffleId", shuffleId) + .add("reduceId", reduceId) + .toString(); + } + + String generateFileName() { + return String.format("mergedShuffle_%s_%d_%d.data", appId, shuffleId, reduceId); + } + + String generateIndexFileName() { + return String.format("mergedShuffle_%s_%d_%d.index", appId, shuffleId, reduceId); + } + + boolean compareAppShuffleId(String appId, int shuffleId) { + return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId; + } + } + + /** + * Metadata tracked for an actively merged shuffle partition + */ + public static class AppShufflePartitionInfo { + // The merged shuffle data file + final File targetFile; + public final FileChannel channel; + // Location offset of the last successfully merged block for this shuffle partition + private long position; + // Indicating whether failure was encountered when merging the previous block + private boolean encounteredFailure; + // Track the map Id whose block is being merged for this shuffle partition + private int currentMapId; + // Bitmap tracking which mapper's blocks have been merged for this shuffle partition + private RoaringBitmap mapTracker; + // The merged shuffle index file + private final FileChannel indexChannel; + private final DataOutputStream indexWriteStream; + // The offset for the last chunk tracked in the index file for this shuffle partition + private long lastChunkOffset; + + AppShufflePartitionInfo(File targetFile, File indexFile) throws IOException { + targetFile.createNewFile(); + this.targetFile = targetFile; + this.channel = new FileOutputStream(targetFile, true).getChannel(); + indexFile.createNewFile(); + FileOutputStream fos = new FileOutputStream(indexFile, true); + indexChannel = fos.getChannel(); + this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); + // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() + updateLastChunkOffset(0L); + this.position = 0; + this.encounteredFailure = false; + this.currentMapId = -1; + this.mapTracker = new RoaringBitmap(); + } + + public long getPosition() { + return position; + } + + public void setPosition(long position) { + this.position = position; + } + + boolean isEncounteredFailure() { + return encounteredFailure; + } + + void setEncounteredFailure(boolean encounteredFailure) { + this.encounteredFailure = encounteredFailure; + } + + int getCurrentMapId() { + return currentMapId; + } + + void setCurrentMapId(int mapId) { + this.currentMapId = mapId; + } + + long getLastChunkOffset() { + return lastChunkOffset; + } + + void blockMerged(int mapId) { + mapTracker.add(mapId); + } + + void updateLastChunkOffset(long lastChunkOffset) throws IOException { + long startPos = indexChannel.position(); + try { + indexWriteStream.writeLong(lastChunkOffset); + } catch(IOException ioe) { + // reset the position to avoid corrupting index files during exception. + indexChannel.position(startPos); + throw ioe; + } + this.lastChunkOffset = lastChunkOffset; + } + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java new file mode 100644 index 0000000000000..349c150ef1455 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java @@ -0,0 +1,86 @@ +/* + * 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 io.netty.buffer.ByteBuf; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.spark.network.protocol.Encoders; + +/** Base class for fetch shuffle blocks and chunks. */ +public abstract class AbstractFetchShuffleBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final int shuffleId; + + protected AbstractFetchShuffleBlocks( + String appId, + String execId, + int shuffleId) { + this.appId = appId; + this.execId = execId; + this.shuffleId = shuffleId; + } + + public ToStringBuilder toStringHelper() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("shuffleId", shuffleId); + } + + /** + * Returns number of blocks in the request. + */ + public abstract int getNumBlocks(); + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AbstractFetchShuffleBlocks that = (AbstractFetchShuffleBlocks) o; + if (shuffleId != that.shuffleId) return false; + if (!appId.equals(that.appId)) return false; + if (!execId.equals(that.execId)) return false; + return true; + } + + @Override + public int hashCode() { + int result = appId.hashCode(); + result = 31 * result + execId.hashCode(); + result = 31 * result + shuffleId; + return result; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + 4; /* encoded length of shuffleId */ + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + buf.writeInt(shuffleId); + } +} 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 89d8dfe8716b8..b4a7aed819444 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 @@ -47,7 +47,10 @@ public abstract class BlockTransferMessage implements Encodable { public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), - FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11); + FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11), + PUSH_BLOCK_STREAM(12), FINALIZE_SHUFFLE_MERGE(13), MERGE_STATUSES(14), + FETCH_MERGED_BLOCKS_META(15), MERGED_BLOCKS_META(16), + FETCH_SHUFFLE_BLOCK_CHUNKS(17); private final byte id; @@ -78,6 +81,12 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 9: return FetchShuffleBlocks.decode(buf); case 10: return GetLocalDirsForExecutors.decode(buf); case 11: return LocalDirsForExecutors.decode(buf); + case 12: return PushBlockStream.decode(buf); + case 13: return FinalizeShuffleMerge.decode(buf); + case 14: return MergeStatuses.decode(buf); + case 15: return FetchMergedBlocksMeta.decode(buf); + case 16: return MergedBlocksMeta.decode(buf); + case 17: return FetchShuffleBlockChunks.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/FetchMergedBlocksMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java new file mode 100644 index 0000000000000..863aa1bfae774 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java @@ -0,0 +1,83 @@ +/* + * 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 java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** + * Request to find the meta information for the specified merged blocks. The meta information + * currently contains only the number of chunks in each merged blocks. + */ +public class FetchMergedBlocksMeta extends BlockTransferMessage { + public final String appId; + public final String[] blockIds; + + public FetchMergedBlocksMeta(String appId, String[] blockIds) { + this.appId = appId; + this.blockIds = blockIds; + } + + @Override + protected Type type() { return Type.FETCH_MERGED_BLOCKS_META; } + + @Override + public int hashCode() { + return appId.hashCode() * 41 + Arrays.hashCode(blockIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("blockIds", Arrays.toString(blockIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof FetchMergedBlocksMeta) { + FetchMergedBlocksMeta o = (FetchMergedBlocksMeta) other; + return Objects.equal(appId, o.appId) + && Arrays.equals(blockIds, o.blockIds); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.StringArrays.encodedLength(blockIds); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.StringArrays.encode(buf, blockIds); + } + + public static FetchMergedBlocksMeta decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String[] blockIds = Encoders.StringArrays.decode(buf); + return new FetchMergedBlocksMeta(appId, blockIds); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java new file mode 100644 index 0000000000000..6e867a7e89803 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java @@ -0,0 +1,123 @@ +/* + * 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 java.util.Arrays; + +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + + +/** Request to read a set of block chunks. Returns {@link StreamHandle}. */ +public class FetchShuffleBlockChunks extends AbstractFetchShuffleBlocks { + // The length of reduceIds must equal to chunkIds.size(). + // The i-th int[] in chunkIds contains all the chunks for the i-th reduceId in reduceIds. + public final int[] reduceIds; + public final int[][] chunkIds; + + public FetchShuffleBlockChunks( + String appId, + String execId, + int shuffleId, + int[] reduceIds, + int[][] chunkIds) { + super(appId, execId, shuffleId); + this.reduceIds = reduceIds; + this.chunkIds = chunkIds; + assert(reduceIds.length == chunkIds.length); + } + + @Override + protected Type type() { return Type.FETCH_SHUFFLE_BLOCK_CHUNKS; } + + @Override + public String toString() { + return toStringHelper() + .append("reduceIds", Arrays.toString(reduceIds)) + .append("chunkIds", Arrays.deepToString(chunkIds)) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + FetchShuffleBlockChunks that = (FetchShuffleBlockChunks) o; + if (!super.equals(that)) return false; + if (!Arrays.equals(reduceIds, that.reduceIds)) return false; + return Arrays.deepEquals(chunkIds, that.chunkIds); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + Arrays.hashCode(reduceIds); + result = 31 * result + Arrays.deepHashCode(chunkIds); + return result; + } + + @Override + public int encodedLength() { + int encodedLengthOfChunkIds = 0; + for (int[] ids: chunkIds) { + encodedLengthOfChunkIds += Encoders.IntArrays.encodedLength(ids); + } + return super.encodedLength() + + Encoders.IntArrays.encodedLength(reduceIds) + + 4 /* encoded length of chunkIds.size() */ + + encodedLengthOfChunkIds; + } + + @Override + public void encode(ByteBuf buf) { + super.encode(buf); + Encoders.IntArrays.encode(buf, reduceIds); + buf.writeInt(chunkIds.length); + for (int[] ids: chunkIds) { + Encoders.IntArrays.encode(buf, ids); + } + } + + @Override + public int getNumBlocks() { + int numBlocks = 0; + for (int[] ids : chunkIds) { + numBlocks += ids.length; + } + return numBlocks; + } + + public static FetchShuffleBlockChunks decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + int shuffleId = buf.readInt(); + int[] reduceIds = Encoders.IntArrays.decode(buf); + int chunkIdsLen = buf.readInt(); + int[][] chunkIds = new int[chunkIdsLen][]; + for (int i = 0; i < chunkIdsLen; i++) { + chunkIds[i] = Encoders.IntArrays.decode(buf); + } + return new FetchShuffleBlockChunks( + appId, execId, shuffleId, reduceIds, chunkIds); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index 98057d58f7ab5..f1bf9927be0ee 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -29,10 +29,7 @@ import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** Request to read a set of blocks. Returns {@link StreamHandle}. */ -public class FetchShuffleBlocks extends BlockTransferMessage { - public final String appId; - public final String execId; - public final int shuffleId; +public class FetchShuffleBlocks extends AbstractFetchShuffleBlocks { // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. public final long[] mapIds; @@ -50,9 +47,7 @@ public FetchShuffleBlocks( long[] mapIds, int[][] reduceIds, boolean batchFetchEnabled) { - this.appId = appId; - this.execId = execId; - this.shuffleId = shuffleId; + super(appId, execId, shuffleId); this.mapIds = mapIds; this.reduceIds = reduceIds; assert(mapIds.length == reduceIds.length); @@ -69,10 +64,7 @@ public FetchShuffleBlocks( @Override public String toString() { - return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) - .append("appId", appId) - .append("execId", execId) - .append("shuffleId", shuffleId) + return toStringHelper() .append("mapIds", Arrays.toString(mapIds)) .append("reduceIds", Arrays.deepToString(reduceIds)) .append("batchFetchEnabled", batchFetchEnabled) @@ -85,35 +77,40 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; FetchShuffleBlocks that = (FetchShuffleBlocks) o; - - if (shuffleId != that.shuffleId) return false; + if (!super.equals(that)) return false; if (batchFetchEnabled != that.batchFetchEnabled) return false; - if (!appId.equals(that.appId)) return false; - if (!execId.equals(that.execId)) return false; if (!Arrays.equals(mapIds, that.mapIds)) return false; return Arrays.deepEquals(reduceIds, that.reduceIds); } @Override public int hashCode() { - int result = appId.hashCode(); - result = 31 * result + execId.hashCode(); - result = 31 * result + shuffleId; + int result = super.hashCode(); result = 31 * result + Arrays.hashCode(mapIds); result = 31 * result + Arrays.deepHashCode(reduceIds); result = 31 * result + (batchFetchEnabled ? 1 : 0); return result; } + @Override + public int getNumBlocks() { + if (batchFetchEnabled) { + return mapIds.length; + } + int numBlocks = 0; + for (int[] ids : reduceIds) { + numBlocks += ids.length; + } + return numBlocks; + } + @Override public int encodedLength() { int encodedLengthOfReduceIds = 0; for (int[] ids: reduceIds) { encodedLengthOfReduceIds += Encoders.IntArrays.encodedLength(ids); } - return Encoders.Strings.encodedLength(appId) - + Encoders.Strings.encodedLength(execId) - + 4 /* encoded length of shuffleId */ + return super.encodedLength() + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds @@ -122,9 +119,7 @@ public int encodedLength() { @Override public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - Encoders.Strings.encode(buf, execId); - buf.writeInt(shuffleId); + super.encode(buf); Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java new file mode 100644 index 0000000000000..8c9dd8aebba17 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java @@ -0,0 +1,85 @@ +/* + * 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; + + +/** + * Request to notify external shuffle service to finalize merge for a given shuffle. + * Returns {@link MergeStatuses} + */ +public class FinalizeShuffleMerge extends BlockTransferMessage { + public final String appId; + public final int shuffleId; + + public FinalizeShuffleMerge( + String appId, + int shuffleId) { + this.appId = appId; + this.shuffleId = shuffleId; + } + + @Override + protected BlockTransferMessage.Type type() { + return Type.FINALIZE_SHUFFLE_MERGE; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, shuffleId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("shuffleId", shuffleId) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof FinalizeShuffleMerge) { + FinalizeShuffleMerge o = (FinalizeShuffleMerge) other; + return Objects.equal(appId, o.appId) + && shuffleId == o.shuffleId; + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + buf.writeInt(shuffleId); + } + + public static FinalizeShuffleMerge decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + int shuffleId = buf.readInt(); + return new FinalizeShuffleMerge(appId, shuffleId); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java new file mode 100644 index 0000000000000..ab16e9450f874 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java @@ -0,0 +1,119 @@ +/* + * 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 java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.roaringbitmap.RoaringBitmap; + +import org.apache.spark.network.protocol.Encoders; + + +/** + * Result returned by an ExternalShuffleService to a scheduler. This represents the result + * of all the remote shuffle block merge operations performed by an ExternalShuffleService + * for a given shuffle ID. It includes the shuffle ID, an array of bitmaps each representing + * the set of mapper partition blocks that are merged for a given reducer partition, an array + * of reducer IDs, and an array of merged shuffle partition sizes. The 3 arrays list information + * about all the reducer partitions merged by the ExternalShuffleService in the same order. + */ +public class MergeStatuses extends BlockTransferMessage { + /** Shuffle ID **/ + public final int shuffleId; + /** + * Array of bitmaps tracking the set of mapper partition blocks merged for each + * reducer partition + */ + public final RoaringBitmap[] bitmaps; + /** Array of reducer IDs **/ + public final int[] reduceIds; + /** + * Array of merged shuffle partition block size. Each represents the total size of all + * merged shuffle partition blocks for one reducer partition. + * **/ + public final long[] sizes; + + public MergeStatuses( + int shuffleId, + RoaringBitmap[] bitmaps, + int[] reduceIds, + long[] sizes) { + this.shuffleId = shuffleId; + this.bitmaps = bitmaps; + this.reduceIds = reduceIds; + this.sizes = sizes; + } + + @Override + protected Type type() { + return Type.MERGE_STATUSES; + } + + @Override + public int hashCode() { + int objectHashCode = Objects.hashCode(shuffleId); + return (objectHashCode * 41 + Arrays.hashCode(reduceIds) * 41 + + Arrays.hashCode(bitmaps) * 41 + Arrays.hashCode(sizes)); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("shuffleId", shuffleId) + .add("reduceId size", reduceIds.length) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof MergeStatuses) { + MergeStatuses o = (MergeStatuses) other; + return Objects.equal(shuffleId, o.shuffleId) + && Arrays.equals(bitmaps, o.bitmaps) + && Arrays.equals(reduceIds, o.reduceIds) + && Arrays.equals(sizes, o.sizes); + } + return false; + } + + @Override + public int encodedLength() { + return 4 // int + + Encoders.BitmapArrays.encodedLength(bitmaps) + + Encoders.IntArrays.encodedLength(reduceIds) + + Encoders.LongArrays.encodedLength(sizes); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeInt(shuffleId); + Encoders.BitmapArrays.encode(buf, bitmaps); + Encoders.IntArrays.encode(buf, reduceIds); + Encoders.LongArrays.encode(buf, sizes); + } + + public static MergeStatuses decode(ByteBuf buf) { + int shuffleId = buf.readInt(); + RoaringBitmap[] bitmaps = Encoders.BitmapArrays.decode(buf); + int[] reduceIds = Encoders.IntArrays.decode(buf); + long[] sizes = Encoders.LongArrays.decode(buf); + return new MergeStatuses(shuffleId, bitmaps, reduceIds, sizes); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java new file mode 100644 index 0000000000000..94c3e616491f0 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java @@ -0,0 +1,75 @@ +/* + * 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 java.util.Arrays; +import javax.annotation.Nonnull; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** + * Response of {@link FetchMergedBlocksMeta}. + */ +public class MergedBlocksMeta extends BlockTransferMessage { + + public final int[] numChunks; + + public MergedBlocksMeta(@Nonnull int[] numChunks) { + this.numChunks = numChunks; + } + + @Override + protected Type type() { return Type.MERGED_BLOCKS_META; } + + @Override + public int hashCode() { + return Arrays.hashCode(numChunks); + } + + @Override + public String toString() { + Objects.ToStringHelper helper = Objects.toStringHelper(this); + return helper.add("numChunks", Arrays.toString(numChunks)).toString(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof MergedBlocksMeta) { + MergedBlocksMeta o = (MergedBlocksMeta) other; + return Arrays.equals(numChunks, o.numChunks); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.IntArrays.encodedLength(numChunks); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.IntArrays.encode(buf, numChunks); + } + + public static MergedBlocksMeta decode(ByteBuf buf) { + return new MergedBlocksMeta(Encoders.IntArrays.decode(buf)); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java new file mode 100644 index 0000000000000..083b17aebe179 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -0,0 +1,92 @@ +/* + * 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; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + + +/** + * Request to push a block to a remote shuffle service to be merged in push based shuffle. + */ +public class PushBlockStream extends BlockTransferMessage { + public final String appId; + public final String blockId; + public final int index; + + public PushBlockStream(String appId, String blockId, int index) { + this.appId = appId; + this.blockId = blockId; + this.index = index; + } + + @Override + protected Type type() { + return Type.PUSH_BLOCK_STREAM; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, blockId, index); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("blockId", blockId) + .add("index", index) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof PushBlockStream) { + PushBlockStream o = (PushBlockStream) other; + return Objects.equal(appId, o.appId) + && Objects.equal(blockId, o.blockId) + && index == o.index; + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(blockId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, blockId); + buf.writeInt(index); + } + + public static PushBlockStream decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String blockId = Encoders.Strings.decode(buf); + int index = buf.readInt(); + return new PushBlockStream(appId, blockId, index); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 455351fcf767c..ddbddba4db0cb 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.network.shuffle; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Iterator; @@ -25,6 +26,7 @@ import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.roaringbitmap.RoaringBitmap; import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.any; @@ -38,7 +40,12 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.FetchMergedBlocksMeta; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.MergedBlocksMeta; +import org.apache.spark.network.shuffle.protocol.MergeStatuses; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.shuffle.protocol.StreamHandle; @@ -50,6 +57,7 @@ public class ExternalBlockHandlerSuite { OneForOneStreamManager streamManager; ExternalShuffleBlockResolver blockResolver; RpcHandler handler; + MergedShuffleFileManager mergedShuffleManager; ManagedBuffer[] blockMarkers = { new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) @@ -59,7 +67,8 @@ public class ExternalBlockHandlerSuite { public void beforeEach() { streamManager = mock(OneForOneStreamManager.class); blockResolver = mock(ExternalShuffleBlockResolver.class); - handler = new ExternalBlockHandler(streamManager, blockResolver); + mergedShuffleManager = mock(MergedShuffleFileManager.class); + handler = new ExternalBlockHandler(streamManager, blockResolver, mergedShuffleManager); } @Test @@ -222,4 +231,143 @@ public void testBadMessages() { verify(callback, never()).onSuccess(any(ByteBuffer.class)); verify(callback, never()).onFailure(any(Throwable.class)); } + + @Test + public void testFetchMergedBlocksMeta() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + when(mergedShuffleManager.getChunkCount("app0", 0, 0)).thenReturn(1); + when(mergedShuffleManager.getChunkCount("app0", 0, 1)).thenReturn(3); + when(mergedShuffleManager.getChunkCount("app0", 0, 2)).thenReturn(5); + + ByteBuffer chunksCountsReq = new FetchMergedBlocksMeta("app0", + new String[]{"shuffle_0_-1_0", "shuffle_0_-1_1", "shuffle_0_-1_2"}).toByteBuffer(); + handler.receive(client, chunksCountsReq, callback); + for (int reduceId = 0; reduceId < 3; reduceId++) { + verify(mergedShuffleManager, times(1)).getChunkCount("app0", 0, reduceId); + } + ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure(any()); + + MergedBlocksMeta chunkCounts = + (MergedBlocksMeta) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + + assertNotNull(chunkCounts.numChunks); + assertEquals(3, chunkCounts.numChunks.length); + assertArrayEquals(new int[]{1, 3, 5}, chunkCounts.numChunks); + } + + @Test + public void testFinalizeShuffleMerge() throws IOException { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + FinalizeShuffleMerge req = new FinalizeShuffleMerge("app0", 0); + RoaringBitmap bitmap = RoaringBitmap.bitmapOf(0, 1, 2); + MergeStatuses statuses = new MergeStatuses(0, new RoaringBitmap[]{bitmap}, + new int[]{3}, new long[]{30}); + when(mergedShuffleManager.finalizeShuffleMerge(req)).thenReturn(statuses); + + ByteBuffer reqBuf = req.toByteBuffer(); + handler.receive(client, reqBuf, callback); + verify(mergedShuffleManager, times(1)).finalizeShuffleMerge(req); + ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure(any()); + + MergeStatuses mergeStatuses = + (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + assertEquals(mergeStatuses, statuses); + } + + @Test + public void testOpenBlocksWithShuffleChunks() { + verifyBlockChunkFetches(true); + } + + @Test + public void testFetchShuffleChunks() { + verifyBlockChunkFetches(false); + } + + private void verifyBlockChunkFetches(boolean useOpenBlocks) { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + ByteBuffer buffer; + if (useOpenBlocks) { + OpenBlocks openBlocks = + new OpenBlocks( + "app0", + "exec1", + new String[] { + "shuffleChunk_0_0_0", + "shuffleChunk_0_0_1", + "shuffleChunk_0_1_0", + "shuffleChunk_0_1_1" + }); + buffer = openBlocks.toByteBuffer(); + } else { + FetchShuffleBlockChunks fetchChunks = + new FetchShuffleBlockChunks( + "app0", "exec1", 0, new int[] {0, 1}, new int[][] {{0, 1}, {0, 1}}); + buffer = fetchChunks.toByteBuffer(); + } + ManagedBuffer[][] buffers = + new ManagedBuffer[][] { + { + new NioManagedBuffer(ByteBuffer.wrap(new byte[5])), + new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) + }, + { + new NioManagedBuffer(ByteBuffer.wrap(new byte[5])), + new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) + } + }; + for (int reduceId = 0; reduceId < 2; reduceId++) { + for (int chunkId = 0; chunkId < 2; chunkId++) { + when(mergedShuffleManager.getMergedBlockData("app0", 0, reduceId, chunkId)).thenReturn(buffers[reduceId][chunkId]); + } + } + + handler.receive(client, buffer, callback); + + ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure(any()); + + StreamHandle handle = + (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + assertEquals(4, handle.numChunks); + + @SuppressWarnings("unchecked") + ArgumentCaptor> stream = (ArgumentCaptor>) + (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); + verify(streamManager, times(1)).registerStream(any(), stream.capture(), any()); + + Iterator bufferIter = stream.getValue(); + for (int reduceId = 0; reduceId < 2; reduceId++) { + for (int chunkId = 0; chunkId < 2; chunkId++) { + assertEquals(buffers[reduceId][chunkId], bufferIter.next()); + } + } + assertFalse(bufferIter.hasNext()); + + verify(mergedShuffleManager, never()).getChunkCount(anyString(), anyInt(), anyInt()); + verify(blockResolver, never()).getBlockData( + anyString(), anyString(), anyInt(), anyInt(), anyInt()); + verify(mergedShuffleManager, times(1)).getMergedBlockData("app0", 0, 0, 0); + verify(mergedShuffleManager, times(1)).getMergedBlockData("app0", 0, 0, 1); + + // Verify open block request latency metrics + Timer openBlockRequestLatencyMillis = (Timer) ((ExternalBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("openBlockRequestLatencyMillis"); + assertEquals(1, openBlockRequestLatencyMillis.getCount()); + // Verify block transfer metrics + Meter blockTransferRateBytes = (Meter) ((ExternalBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("blockTransferRateBytes"); + assertEquals(24, blockTransferRateBytes.getCount()); + } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 285eedb39c65c..eb021c32b78de 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -46,6 +46,7 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.MapConfigProvider; @@ -201,6 +202,27 @@ public void testEmptyBlockFetch() { } } + @Test + public void testShuffleBlockChunksFetch() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffleChunk_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shuffleChunk_0_0_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("shuffleChunk_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlockChunks("app-id", "exec-id", 0, + new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), + conf); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess( + "shuffleChunk_0_0_" + i, blocks.get("shuffleChunk_0_0_" + i)); + } + } + /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java new file mode 100644 index 0000000000000..7c1e408bc5ebb --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -0,0 +1,157 @@ +/* + * 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; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; + +import com.google.common.collect.Maps; +import io.netty.buffer.Unpooled; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; +import org.apache.spark.network.util.JavaUtils; + + +public class OneForOneBlockPusherSuite { + + @Test + public void testPushOne() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "shuffle_0_0_0", 0))); + + verify(listener).onBlockFetchSuccess(eq("shuffle_0_0_0"), any()); + } + + @Test + public void testPushThree() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess(eq("b" + i), any()); + } + } + + @Test + public void testServerFailures() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + } + + @Test + public void testServerClientFailures() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("b2", null); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = pushBlocks( + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); + + verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b0"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + } + + /** + * Begins a push on the given set of blocks by mocking the response from server side. + * If a block is an empty byte, a server side exception will be thrown. + * If a block is null, a client side exception will be thrown. + */ + private static BlockFetchingListener pushBlocks( + LinkedHashMap blocks, + String[] blockIds, + Iterable expectMessages) { + TransportClient client = mock(TransportClient.class); + BlockFetchingListener listener = mock(BlockFetchingListener.class); + OneForOneBlockPusher pusher = + new OneForOneBlockPusher(client, "app-id", blockIds, listener, blocks); + + Iterator> blockIterator = blocks.entrySet().iterator(); + Iterator msgIterator = expectMessages.iterator(); + doAnswer(invocation -> { + ByteBuffer header = ((ManagedBuffer) invocation.getArguments()[0]).nioByteBuffer(); + BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteBuffer(header); + RpcResponseCallback callback = (RpcResponseCallback) invocation.getArguments()[2]; + Map.Entry entry = blockIterator.next(); + ManagedBuffer block = entry.getValue(); + if (block != null && block.nioByteBuffer().capacity() > 0) { + callback.onSuccess(header); + } else if (block != null) { + callback.onFailure(new RuntimeException(JavaUtils.encodeHeaderIntoErrorString(header, + new RuntimeException("Failed " + entry.getKey())))); + } else { + callback.onFailure(new RuntimeException("Quick fail " + entry.getKey())); + } + assertEquals(msgIterator.next(), message); + return null; + }).when(client).uploadStream(any(ManagedBuffer.class), any(), any(RpcResponseCallback.class)); + + pusher.start(); + return listener; + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java new file mode 100644 index 0000000000000..a0d227bb8b1c5 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -0,0 +1,152 @@ +/* + * 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; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Paths; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.*; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.client.StreamCallbackWithID; +import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class RemoteBlockPushResolverSuite { + + private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); + private static final String[] LOCAL_DIRS = new String[]{"target/l1", "target/l2"}; + private static final String TEST_APP = "testApp"; + + private TransportConf conf; + private RemoteBlockPushResolver pushResolver; + + @Before + public void before() throws IOException { + cleanupLocalDirs(); + for (String localDir : LOCAL_DIRS) { + Files.createDirectories(Paths.get(localDir).resolve(TEST_APP)); + } + MapConfigProvider provider = new MapConfigProvider( + ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); + conf = new TransportConf("shuffle", provider); + pushResolver = new RemoteBlockPushResolver(conf, LOCAL_DIRS); + pushResolver.registerApplication(TEST_APP, TEST_APP); + } + + @After + public void after() { + try { + pushResolver.applicationRemoved(TEST_APP, true); + cleanupLocalDirs(); + } catch (IOException e) { + // don't fail if clean up doesn't succeed. + log.warn("Error deleting test local dirs", e); + } + } + + private static void cleanupLocalDirs() throws IOException { + for (String local : LOCAL_DIRS) { + FileUtils.deleteDirectory(new File(local)); + } + } + + @Test(expected = RuntimeException.class) + public void testNoIndexFile() { + try { + pushResolver.getChunkCount(TEST_APP, 0, 0); + } catch (Throwable t) { + assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); + Throwables.propagate(t); + } + } + + @Test + public void testChunkCountsAndBlockData() throws IOException { + PushBlockStream[] pushBlocks = new PushBlockStream[] { + new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0), + new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0), + }; + ByteBuffer[] blocks = new ByteBuffer[]{ + ByteBuffer.wrap(new byte[4]), + ByteBuffer.wrap(new byte[5]) + }; + pushBlockHelper(pushBlocks, blocks); + int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + assertEquals(2, numChunks); + validateChunks(0, 0, numChunks, new int[]{4, 5}); + } + + @Test + public void testMultipleBlocksInAChunk() throws IOException { + PushBlockStream[] pushBlocks = new PushBlockStream[] { + new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0), + new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0), + new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0), + new PushBlockStream(TEST_APP, "shuffle_0_3_0", 0), + }; + ByteBuffer[] buffers = new ByteBuffer[]{ + ByteBuffer.wrap(new byte[2]), + ByteBuffer.wrap(new byte[3]), + ByteBuffer.wrap(new byte[5]), + ByteBuffer.wrap(new byte[3]) + }; + pushBlockHelper(pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + assertEquals(3, numChunks); + validateChunks(0, 0, numChunks, new int[]{5, 5, 3}); + } + + private void validateChunks( + int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { + for (int i = 0; i < numChunks; i++) { + FileSegmentManagedBuffer mb = + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, + i); + assertEquals(expectedSizes[i], mb.getLength()); + } + } + + private void pushBlockHelper(PushBlockStream[] pushBlocks, ByteBuffer[] blocks) + throws IOException { + Preconditions.checkArgument(pushBlocks.length == blocks.length); + for (int i = 0; i < pushBlocks.length; i++) { + StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( + new PushBlockStream(TEST_APP, pushBlocks[i].blockId, 0)); + stream.onData(stream.getID(), blocks[i]); + stream.onComplete(stream.getID()); + } + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java new file mode 100644 index 0000000000000..a26c60844b5ae --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java @@ -0,0 +1,42 @@ +/* + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Assert; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class FetchShuffleBlockChunksSuite { + + @Test + public void testFetchShuffleBlockChunksEncodeDecode() { + FetchShuffleBlockChunks shuffleBlockChunks = + new FetchShuffleBlockChunks("app0", "exec1", 0, new int[] {0}, new int[][] {{0, 1}}); + Assert.assertEquals(2, shuffleBlockChunks.getNumBlocks()); + int len = shuffleBlockChunks.encodedLength(); + Assert.assertEquals(45, len); + ByteBuf buf = Unpooled.buffer(len); + shuffleBlockChunks.encode(buf); + + FetchShuffleBlockChunks decoded = FetchShuffleBlockChunks.decode(buf); + assertEquals(shuffleBlockChunks, decoded); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java new file mode 100644 index 0000000000000..e3f6d252208a1 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java @@ -0,0 +1,42 @@ +/* + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Assert; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class FetchShuffleBlocksSuite { + + @Test + public void testFetchShuffleBlockEncodeDecode() { + FetchShuffleBlocks fetchShuffleBlocks = + new FetchShuffleBlocks("app0", "exec1", 0, new long[] {0}, new int[][] {{0, 1}}, false); + Assert.assertEquals(2, fetchShuffleBlocks.getNumBlocks()); + int len = fetchShuffleBlocks.encodedLength(); + Assert.assertEquals(50, len); + ByteBuf buf = Unpooled.buffer(len); + fetchShuffleBlocks.encode(buf); + + FetchShuffleBlocks decoded = FetchShuffleBlocks.decode(buf); + assertEquals(fetchShuffleBlocks, decoded); + } +} diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 3d14318bf90f0..9f19b205ecb45 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -54,6 +55,7 @@ import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalBlockHandler; +import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.TransportConf; import org.apache.spark.network.yarn.util.HadoopConfigProvider; @@ -94,6 +96,12 @@ public class YarnShuffleService extends AuxiliaryService { static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure"; private static final boolean DEFAULT_STOP_ON_FAILURE = false; + // Used by shuffle merge manager to create merged shuffle files. + private static final String YARN_LOCAL_DIRS = "yarn.nodemanager.local-dirs"; + private static final String MERGE_MANAGER_DIR = "merge_manager"; + protected static final String MERGE_DIR_RELATIVE_PATH = + "usercache/%s/appcache/%s/" + MERGE_MANAGER_DIR; + // just for testing when you want to find an open port @VisibleForTesting static int boundPort = -1; @@ -126,6 +134,7 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; + private RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -172,7 +181,10 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile); + String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)) + .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); + shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); + blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests @@ -276,6 +288,8 @@ public void initializeApplication(ApplicationInitializationContext context) { } secretManager.registerApp(appId, shuffleSecret); } + shuffleMergeManager.registerApplication( + appId, String.format(MERGE_DIR_RELATIVE_PATH, context.getUser(), appId)); } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } @@ -297,6 +311,9 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); + // TODO change cleanupLocalDirs to false. These should be deleted by yarn when the app + // finishes. + shuffleMergeManager.applicationRemoved(appId, true); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala index d681c13337e0d..4ce46156c0a92 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala @@ -61,7 +61,9 @@ class ExternalShuffleServiceMetricsSuite extends SparkFunSuite { "registeredExecutorsSize", "registerExecutorRequestLatencyMillis", "shuffle-server.usedDirectMemory", - "shuffle-server.usedHeapMemory") + "shuffle-server.usedHeapMemory", + "finalizeShuffleMergeLatencyMillis", + "fetchMergedBlocksMetaLatencyMillis") ) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala index 63ac1af8a9127..8bfc1a0d3ff58 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala @@ -40,7 +40,8 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { val allMetrics = Set( "openBlockRequestLatencyMillis", "registerExecutorRequestLatencyMillis", "blockTransferRateBytes", "registeredExecutorsSize", "numActiveConnections", - "numCaughtExceptions") + "numCaughtExceptions", "fetchMergedBlocksMetaLatencyMillis", + "finalizeShuffleMergeLatencyMillis") metrics.getMetrics.keySet().asScala should be (allMetrics) } From 3bb084ad89fa604f4dc7032d716b7deb50f758ce Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Mon, 11 May 2020 13:40:32 -0700 Subject: [PATCH 02/29] LIHADOOP-53388 Magnet: Fix a bug with calculating bitmap encoded length RB=2096937 G=spark-reviewers R=chsingh,mshen A=mshen --- .../java/org/apache/spark/network/protocol/Encoders.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index b2111b2f3e237..437d03362117f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -54,13 +54,13 @@ public static String decode(ByteBuf buf) { /** Bitmaps are encoded with their serialization length followed by the serialization bytes. */ public static class Bitmaps { public static int encodedLength(RoaringBitmap b) { + // Compress the bitmap before serializing it + b.trim(); + b.runOptimize(); return 4 + b.serializedSizeInBytes(); } public static void encode(ByteBuf buf, RoaringBitmap b) { - // Compress the bitmap before serializing it - b.trim(); - b.runOptimize(); ByteBuffer outBuffer = ByteBuffer.allocate(b.serializedSizeInBytes()); try { b.serialize(new DataOutputStream(new OutputStream() { From 07815419739b70872e4ba9fe2c2101768059ddb2 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 12 May 2020 15:33:14 -0700 Subject: [PATCH 03/29] LIHADOOP-53438 Using different appId for the tests in RemoteBlockPushResolverSuite RB=2101153 BUG=LIHADOOP-53438 G=spark-reviewers R=mshen,yezhou A=yezhou --- .../shuffle/RemoteBlockPushResolver.java | 3 +- .../shuffle/RemoteBlockPushResolverSuite.java | 77 ++++++++++++------- 2 files changed, 51 insertions(+), 29 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 3b7d0d6148d61..86080fba97305 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -151,7 +151,8 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI File mergedShuffleFile = getMergedShuffleFile(id); if (!mergedShuffleFile.exists()) { throw new RuntimeException( - String.format("Application merged shuffle file is not found (id=%s)", id.toString())); + String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(), + id.toString())); } File indexFile = getMergedIndexFile(id); try { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index a0d227bb8b1c5..0a9b2350ccfdf 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -46,29 +46,25 @@ public class RemoteBlockPushResolverSuite { private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); - private static final String[] LOCAL_DIRS = new String[]{"target/l1", "target/l2"}; - private static final String TEST_APP = "testApp"; private TransportConf conf; private RemoteBlockPushResolver pushResolver; + private String[] localDirs; @Before public void before() throws IOException { + localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(), + Paths.get("target/l2").toAbsolutePath().toString()}; cleanupLocalDirs(); - for (String localDir : LOCAL_DIRS) { - Files.createDirectories(Paths.get(localDir).resolve(TEST_APP)); - } MapConfigProvider provider = new MapConfigProvider( ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); conf = new TransportConf("shuffle", provider); - pushResolver = new RemoteBlockPushResolver(conf, LOCAL_DIRS); - pushResolver.registerApplication(TEST_APP, TEST_APP); + pushResolver = new RemoteBlockPushResolver(conf, localDirs); } @After public void after() { try { - pushResolver.applicationRemoved(TEST_APP, true); cleanupLocalDirs(); } catch (IOException e) { // don't fail if clean up doesn't succeed. @@ -76,8 +72,8 @@ public void after() { } } - private static void cleanupLocalDirs() throws IOException { - for (String local : LOCAL_DIRS) { + private void cleanupLocalDirs() throws IOException { + for (String local : localDirs) { FileUtils.deleteDirectory(new File(local)); } } @@ -85,7 +81,10 @@ private static void cleanupLocalDirs() throws IOException { @Test(expected = RuntimeException.class) public void testNoIndexFile() { try { - pushResolver.getChunkCount(TEST_APP, 0, 0); + String appId = "app_NoIndexFile"; + registerApplication(appId); + pushResolver.getChunkCount(appId, 0, 0); + removeApplication(appId); } catch (Throwable t) { assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); Throwables.propagate(t); @@ -94,27 +93,32 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { + String appId = "app_ChunkCountsAndBlockData"; + registerApplication(appId); PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0), - new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), }; ByteBuffer[] blocks = new ByteBuffer[]{ ByteBuffer.wrap(new byte[4]), ByteBuffer.wrap(new byte[5]) }; - pushBlockHelper(pushBlocks, blocks); - int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + pushBlockHelper(appId, pushBlocks, blocks); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); assertEquals(2, numChunks); - validateChunks(0, 0, numChunks, new int[]{4, 5}); + validateChunks(appId,0, 0, numChunks, new int[]{4, 5}); + removeApplication(appId); } @Test public void testMultipleBlocksInAChunk() throws IOException { + String appId = "app_MultipleBlocksInAChunk"; + registerApplication(appId); PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0), - new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0), - new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0), - new PushBlockStream(TEST_APP, "shuffle_0_3_0", 0), + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_2_0", 0), + new PushBlockStream(appId, "shuffle_0_3_0", 0), }; ByteBuffer[] buffers = new ByteBuffer[]{ ByteBuffer.wrap(new byte[2]), @@ -122,31 +126,48 @@ public void testMultipleBlocksInAChunk() throws IOException { ByteBuffer.wrap(new byte[5]), ByteBuffer.wrap(new byte[3]) }; - pushBlockHelper(pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + pushBlockHelper(appId, pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); assertEquals(3, numChunks); - validateChunks(0, 0, numChunks, new int[]{5, 5, 3}); + validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + removeApplication(appId); + } + + /** + * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. + * This is because when the application gets removed, the directory cleaner removes the merged + * data and file in a different thread which can delete the relevant data of a different test. + */ + private void registerApplication(String appId) throws IOException { + for (String localDir : localDirs) { + Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); + } + pushResolver.registerApplication(appId, appId + "/merge_manager"); + } + + private void removeApplication(String appId) { + pushResolver.applicationRemoved(appId, true); } private void validateChunks( - int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { + String appId, int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { for (int i = 0; i < numChunks; i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } - private void pushBlockHelper(PushBlockStream[] pushBlocks, ByteBuffer[] blocks) + private void pushBlockHelper(String appId, PushBlockStream[] pushBlocks, ByteBuffer[] blocks) throws IOException { Preconditions.checkArgument(pushBlocks.length == blocks.length); for (int i = 0; i < pushBlocks.length; i++) { StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( - new PushBlockStream(TEST_APP, pushBlocks[i].blockId, 0)); + new PushBlockStream(appId, pushBlocks[i].blockId, 0)); stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); } } From 7adf227ff30c39e55e6d26dfce6fb67fc1b724cd Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 14 May 2020 17:36:09 -0700 Subject: [PATCH 04/29] LIHADOOP-53496 Not logging all block push exceptions on the client RB=2104829 BUG=LIHADOOP-53496 G=spark-reviewers R=yezhou,mshen A=mshen --- .../network/shuffle/BlockPushException.java | 3 +++ .../shuffle/RemoteBlockPushResolver.java | 4 ++-- .../network/shuffle/RetryingBlockFetcher.java | 18 ++++++++++++++++-- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java index 567840da7e552..beb3d694bb77d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java @@ -39,6 +39,9 @@ public class BlockPushException extends RuntimeException { public static final String TOO_LATE_MESSAGE_SUFFIX = "received after merged shuffle is finalized"; + public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = + "Couldn't find an opportunity to write block"; + private BlockPushException(PushBlockStream header, String message) { super(message); this.header = header; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 86080fba97305..dd9d22f95403a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -472,8 +472,8 @@ public void onComplete(String streamId) throws IOException { } else { deferredBufs = null; canWrite = false; - throw new RuntimeException(String.format("Couldn't find an opportunity to write " - + "block %s to merged shuffle", msg.blockId)); + throw new RuntimeException(String.format("%s %s to merged shuffle", + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX, msg.blockId)); } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index 6bf3da94030d4..a89d4a807d6d9 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -215,8 +215,11 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { if (shouldRetry(exception)) { initiateRetry(); } else { - logger.error(String.format("Failed to fetch block %s, and will not retry (%s retries)", - blockId, retryCount), exception); + if (shouldLogError(exception)) { + logger.error( + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); + } outstandingBlocksIds.remove(blockId); shouldForwardFailure = true; } @@ -228,5 +231,16 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { listener.onBlockFetchFailure(blockId, exception); } } + + private boolean shouldLogError(Throwable t) { + if ((t.getMessage() != null && t.getMessage() + .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)) || + (t.getCause() != null && t.getCause().getMessage() != null && t.getCause() + .getMessage() + .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX))) { + return false; + } + return true; + } } } From ee75ee946f6bd6336d73010abc0fb84f299fcc53 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Mon, 1 Jun 2020 14:23:33 -0700 Subject: [PATCH 05/29] LIHADOOP-53700 Separate configuration for caching the merged index files in NM RB=2130238 BUG=LIHADOOP-53700 G=spark-reviewers R=mshen,chsingh A=chsingh --- .../apache/spark/network/shuffle/RemoteBlockPushResolver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index dd9d22f95403a..2ce6451f10d23 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -91,7 +91,7 @@ public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); - String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); + String indexCacheSize = conf.get("spark.shuffle.service.mergedIndex.cache.size", "100m"); CacheLoader indexCacheLoader = new CacheLoader() { public ShuffleIndexInformation load(File file) throws IOException { From d1f36c0f97d378fb9eb81015f7e25b4cfcfb0655 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 10 Jun 2020 16:39:15 -0700 Subject: [PATCH 06/29] LIHADOOP-53940 Logging the data file and index file path when shuffle service is unable to create them RB=2146753 BUG=LIHADOOP-53940 G=spark-reviewers R=mshen,yezhou A=mshen,yezhou --- .../network/shuffle/RemoteBlockPushResolver.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 2ce6451f10d23..ebd29e7f0975e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -116,14 +116,18 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // be the first time the merge manager receives a pushed block for a given application // shuffle partition, or after the merged shuffle file is finalized. We handle these // two cases accordingly by checking if the file already exists. + File mergedShuffleFile = getMergedShuffleFile(key); + File mergedIndexFile = getMergedIndexFile(id); try { - File mergedShuffleFile = getMergedShuffleFile(key); if (mergedShuffleFile.exists()) { return null; } else { - return new AppShufflePartitionInfo(mergedShuffleFile, getMergedIndexFile(id)); + return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile); } } catch (IOException e) { + logger.error( + "Cannot create merged shuffle partition {} with shuffle file {} and index file {}", key, + mergedIndexFile.getAbsolutePath(), mergedIndexFile.getAbsolutePath()); throw new RuntimeException(String.format( "Cannot initialize merged shuffle partition %s", key.toString()), e); } @@ -196,7 +200,8 @@ private File getMergedIndexFile(AppShufflePartitionId id) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - Path relativeMergeDir = appsRelativePath.remove(appId); + Path relativeMergeDir = Preconditions.checkNotNull( + appsRelativePath.remove(appId), "application " + appId + " is not registered."); Iterator> iterator = partitions.entrySet().iterator(); while (iterator.hasNext()) { @@ -231,7 +236,7 @@ private void deleteExecutorDirs(Path[] dirs) { logger.debug("Successfully cleaned up directory: {}", localDir); } } catch (Exception e) { - logger.error("Failed to delete directory: " + localDir, e); + logger.error("Failed to delete directory: {}", localDir, e); } } } From aa124b4ecbd7e1d6490ac946b1b9a7a7d8dbfd58 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 15 Jun 2020 13:35:09 -0700 Subject: [PATCH 07/29] LIHADOOP-54059 LIHADOOP-53496 Handle the inconsistencies between local dirs provided to executor and the shuffle service and not log all exceptions at error/warning level RB=2152736 BUG=LIHADOOP-53496,LIHADOOP-54059 G=spark-reviewers R=yezhou,mshen A=mshen --- .../shuffle/RemoteBlockPushResolver.java | 74 +++++++++++++++---- .../shuffle/RemoteBlockPushResolverSuite.java | 35 +++++++-- 2 files changed, 91 insertions(+), 18 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index ebd29e7f0975e..afc57e5a6f361 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -69,7 +69,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); private final Path[] localDirs; - private final ConcurrentMap appsRelativePath; + private final ConcurrentMap appsPathInfo; private final ConcurrentMap partitions; private final Executor directoryCleaner; @@ -86,7 +86,7 @@ public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { this.localDirs[i] = Paths.get(localDirs[i]); } this.partitions = Maps.newConcurrentMap(); - this.appsRelativePath = Maps.newConcurrentMap(); + this.appsPathInfo = Maps.newConcurrentMap(); this.directoryCleaner = Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); @@ -181,10 +181,13 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI // TODO should we use subDirsPerLocalDir to potentially reduce inode size? private File getFile(String appId, String filename) { int hash = JavaUtils.nonNegativeHash(filename); - Path localDir = localDirs[hash % localDirs.length]; - Path relativeMergeDir = Preconditions.checkNotNull( - appsRelativePath.get(appId), "application " + appId + " is not registered."); - return new File(localDir.resolve(relativeMergeDir).toFile(), filename); + // TODO: Change the message when this service is able to handle NM restart + AppPathsInfo appPathsInfo = Preconditions.checkNotNull( + appsPathInfo.get(appId), + "application " + appId + " is not registered or NM was restarted."); + Path[] activeLocalDirs = appPathsInfo.getActiveLocalDirs(localDirs); + Path localDir = activeLocalDirs[hash % activeLocalDirs.length]; + return new File(localDir.resolve(appPathsInfo.relativeMergeDir).toFile(), filename); } private File getMergedShuffleFile(AppShufflePartitionId id) { @@ -200,8 +203,10 @@ private File getMergedIndexFile(AppShufflePartitionId id) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - Path relativeMergeDir = Preconditions.checkNotNull( - appsRelativePath.remove(appId), "application " + appId + " is not registered."); + // TODO: Change the message when this service is able to handle NM restart + AppPathsInfo appPathsInfo = Preconditions.checkNotNull( + appsPathInfo.get(appId), + "application " + appId + " is not registered or NM was restarted."); Iterator> iterator = partitions.entrySet().iterator(); while (iterator.hasNext()) { @@ -219,8 +224,8 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(localDirs) - .map(dir -> dir.resolve(relativeMergeDir)).toArray(Path[]::new); + Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) + .map(dir -> dir.resolve(appPathsInfo.relativeMergeDir)).toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } @@ -484,8 +489,21 @@ public void onComplete(String streamId) throws IOException { } @Override - public void onFailure(String streamId, Throwable cause) throws IOException { - logger.error("Encountered issue when merging shuffle partition block {}", msg, cause); + public void onFailure(String streamId, Throwable throwable) throws IOException { + if ((throwable.getMessage() != null && + (throwable.getMessage().contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || + throwable.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) || + + (throwable.getCause() != null && throwable.getCause().getMessage() != null && + (throwable.getCause().getMessage().contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || + throwable.getCause().getMessage().contains( + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { + logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); + } else { + logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable); + } // Only update partitionInfo if the failure corresponds to a valid request. If the // request is too late, i.e. received after shuffle merge finalize, #onFailure will // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity @@ -549,7 +567,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc @Override public void registerApplication(String appId, String relativeAppPath) { logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); - appsRelativePath.put(appId, Paths.get(relativeAppPath)); + appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); } /** @@ -688,4 +706,34 @@ void updateLastChunkOffset(long lastChunkOffset) throws IOException { this.lastChunkOffset = lastChunkOffset; } } + + /** + * Wraps all the information related to the merge_dir of an application. + */ + private static class AppPathsInfo { + + private final Path relativeMergeDir; + private Path[] activeLocalDirs; + + AppPathsInfo(Path relativeMergeDir) { + this.relativeMergeDir = Preconditions.checkNotNull( + relativeMergeDir, "relative merge directory path cannot be null"); + } + + private Path[] getActiveLocalDirs(Path[] localDirs) { + if (activeLocalDirs != null) { + return activeLocalDirs; + } + synchronized (this) { + activeLocalDirs = Arrays.stream(localDirs) + .filter(rootDir -> rootDir.resolve(relativeMergeDir).toFile().exists()) + .toArray(Path[]::new); + if (activeLocalDirs.length == 0) { + throw new RuntimeException( + "Did not find any active local directories wrt " + relativeMergeDir); + } + } + return activeLocalDirs; + } + } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 0a9b2350ccfdf..fc8f7f8df7f7a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.Arrays; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -82,7 +83,7 @@ private void cleanupLocalDirs() throws IOException { public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; - registerApplication(appId); + registerApplication(appId, localDirs); pushResolver.getChunkCount(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -94,7 +95,7 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId); + registerApplication(appId, localDirs); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -113,7 +114,31 @@ public void testChunkCountsAndBlockData() throws IOException { @Test public void testMultipleBlocksInAChunk() throws IOException { String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId); + registerApplication(appId, localDirs); + PushBlockStream[] pushBlocks = new PushBlockStream[] { + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_2_0", 0), + new PushBlockStream(appId, "shuffle_0_3_0", 0), + }; + ByteBuffer[] buffers = new ByteBuffer[]{ + ByteBuffer.wrap(new byte[2]), + ByteBuffer.wrap(new byte[3]), + ByteBuffer.wrap(new byte[5]), + ByteBuffer.wrap(new byte[3]) + }; + pushBlockHelper(appId, pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); + assertEquals(3, numChunks); + validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + removeApplication(appId); + } + + @Test + public void testAppUsingFewerLocalDirs() throws IOException { + String appId = "app_AppUsingFewerLocalDirs"; + String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); + registerApplication(appId, activeLocalDirs); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -138,8 +163,8 @@ public void testMultipleBlocksInAChunk() throws IOException { * This is because when the application gets removed, the directory cleaner removes the merged * data and file in a different thread which can delete the relevant data of a different test. */ - private void registerApplication(String appId) throws IOException { - for (String localDir : localDirs) { + private void registerApplication(String appId, String[] activeLocalDirs) throws IOException { + for (String localDir : activeLocalDirs) { Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); } pushResolver.registerApplication(appId, appId + "/merge_manager"); From 3f1fb0cb9b5e1345adf6f5f90b8114085be781a4 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 24 Jun 2020 00:27:27 -0700 Subject: [PATCH 08/29] LIHADOOP-54379 Sorting the disks both on shuffle service and executors RB=2166324 BUG=LIHADOOP-54379 G=spark-reviewers R=yezhou,mshen A=mshen --- .../spark/network/shuffle/RemoteBlockPushResolver.java | 6 ++++++ .../org/apache/spark/network/yarn/YarnShuffleService.java | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index afc57e5a6f361..7f5e8aafcd42d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,6 +37,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; @@ -570,6 +571,11 @@ public void registerApplication(String appId, String relativeAppPath) { appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); } + @VisibleForTesting + public Path[] getLocalDirs() { + return localDirs; + } + /** * ID that uniquely identifies a shuffle partition for an application. This is used to key * the metadata tracked for each shuffle partition that's being actively merged. diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 9f19b205ecb45..0ed63576a75fc 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -134,7 +134,8 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; - private RemoteBlockPushResolver shuffleMergeManager; + @VisibleForTesting + RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -181,7 +182,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)) + String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)).sorted() .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); From bfcb0700b628e4da18c6b6d90023f6316948c4f9 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 23 Jun 2020 23:10:07 -0700 Subject: [PATCH 09/29] LIHADOOP-54370 Not to retry on certain exceptions when pushing blocks RB=2166258 BUG=LIHADOOP-54370 G=spark-reviewers R=mshen,yezhou A=mshen --- .../spark/network/shuffle/ErrorHandler.java | 72 +++++++++++++++++++ .../shuffle/ExternalBlockStoreClient.java | 11 ++- .../network/shuffle/RetryingBlockFetcher.java | 29 ++++---- .../network/shuffle/ErrorHandlerSuite.java | 51 +++++++++++++ 4 files changed, 147 insertions(+), 16 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java new file mode 100644 index 0000000000000..805164193c72f --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.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; + +import java.net.ConnectException; + +/** + * Plugs into {@link RetryingBlockFetcher} to further control when an exception should be retried + * and logged. + * Note: {@link RetryingBlockFetcher} will delegate the exception to this handler only when + * - remaining retries < max retries + * - exception is either an IOException or SaslException + */ + +public interface ErrorHandler { + boolean shouldRetryError(Throwable t); + + default boolean shouldLogError(Throwable t) { + return true; + } + + /** + * A no-op error handler instance. + */ + ErrorHandler NOOP_ERROR_HANDLER = t -> true; + + /** + * The error handler for pushing shuffle blocks to remote shuffle services. + */ + class BlockPushErrorHandler implements ErrorHandler { + + @Override + public boolean shouldRetryError(Throwable t) { + // If it is a connection time out or a connection closed exception, no need to retry. + if (t.getCause() != null && t.getCause() instanceof ConnectException) { + return false; + } + // If the block is too late, there is no need to retry it + return (t.getMessage() == null || !t.getMessage() + .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) && (t.getCause() == null + || t.getCause().getMessage() == null || !t.getCause().getMessage() + .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + } + + @Override + public boolean shouldLogError(Throwable t) { + return (t.getMessage() == null || ( + !t.getMessage().contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) + && !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) && ( + t.getCause() == null || t.getCause().getMessage() == null || (!t.getCause() + .getMessage() + .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && !t.getCause() + .getMessage() + .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))); + } + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index ab789d1ad7d35..a5491a4358ced 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -22,16 +22,19 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.shuffle.protocol.*; - import org.apache.spark.network.TransportContext; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.crypto.AuthClientBootstrap; @@ -45,6 +48,9 @@ * (via BlockTransferService), which has the downside of losing the data if we lose the executors. */ public class ExternalBlockStoreClient extends BlockStoreClient { + private static final Logger logger = LoggerFactory.getLogger(ExternalBlockStoreClient.class); + private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); + private final TransportConf conf; private final boolean authEnabled; private final SecretKeyHolder secretKeyHolder; @@ -141,7 +147,8 @@ public void pushBlocks( }; int maxRetries = conf.maxIORetries(); if (maxRetries > 0) { - new RetryingBlockFetcher(conf, blockPushStarter, blockIds, listener).start(); + new RetryingBlockFetcher( + conf, blockPushStarter, blockIds, listener, PUSH_ERROR_HANDLER).start(); } else { blockPushStarter.createAndStart(blockIds, listener); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index a89d4a807d6d9..bbcfb55853760 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -99,11 +99,14 @@ void createAndStart(String[] blockIds, BlockFetchingListener listener) */ private RetryingBlockFetchListener currentListener; + private final ErrorHandler errorHandler; + public RetryingBlockFetcher( TransportConf conf, RetryingBlockFetcher.BlockFetchStarter fetchStarter, String[] blockIds, - BlockFetchingListener listener) { + BlockFetchingListener listener, + ErrorHandler errorHandler) { this.fetchStarter = fetchStarter; this.listener = listener; this.maxRetries = conf.maxIORetries(); @@ -111,6 +114,15 @@ public RetryingBlockFetcher( this.outstandingBlocksIds = Sets.newLinkedHashSet(); Collections.addAll(outstandingBlocksIds, blockIds); this.currentListener = new RetryingBlockFetchListener(); + this.errorHandler = errorHandler; + } + + public RetryingBlockFetcher( + TransportConf conf, + BlockFetchStarter fetchStarter, + String[] blockIds, + BlockFetchingListener listener) { + this(conf, fetchStarter, blockIds, listener, ErrorHandler.NOOP_ERROR_HANDLER); } /** @@ -178,7 +190,7 @@ private synchronized boolean shouldRetry(Throwable e) { boolean isIOException = e instanceof IOException || (e.getCause() != null && e.getCause() instanceof IOException); boolean hasRemainingRetries = retryCount < maxRetries; - return isIOException && hasRemainingRetries; + return isIOException && hasRemainingRetries && errorHandler.shouldRetryError(e); } /** @@ -215,7 +227,7 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { if (shouldRetry(exception)) { initiateRetry(); } else { - if (shouldLogError(exception)) { + if (errorHandler.shouldLogError(exception)) { logger.error( String.format("Failed to fetch block %s, and will not retry (%s retries)", blockId, retryCount), exception); @@ -231,16 +243,5 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { listener.onBlockFetchFailure(blockId, exception); } } - - private boolean shouldLogError(Throwable t) { - if ((t.getMessage() != null && t.getMessage() - .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)) || - (t.getCause() != null && t.getCause().getMessage() != null && t.getCause() - .getMessage() - .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX))) { - return false; - } - return true; - } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java new file mode 100644 index 0000000000000..905a90d9c4cb0 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -0,0 +1,51 @@ +/* + * 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; + +import java.net.ConnectException; + +import org.junit.Test; + +import static org.junit.Assert.*; + +/** + * Test suite for {@link ErrorHandler} + */ +public class ErrorHandlerSuite { + + @Test + public void testPushErrorRetry() { + ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); + assertFalse(handler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))); + assertFalse(handler.shouldRetryError(new RuntimeException(new ConnectException()))); + assertTrue(handler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + assertTrue(handler.shouldRetryError(new Throwable())); + } + + @Test + public void testPushErrorLogging() { + ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); + assert (!handler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))); + assert (!handler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + assert (handler.shouldLogError(new Throwable())); + } +} From 415a7d565ab8fd30b4cdd60cc32d525a71341fbd Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 23 Jul 2020 17:43:50 -0700 Subject: [PATCH 10/29] LIHADOOP-52494 Magnet fallback to origin shuffle blocks when fetch of a shuffle chunk fails RB=2203642 BUG=LIHADOOP-52494 G=spark-reviewers R=yzhou,mshen,vsowrira A=mshen --- .../network/client/BaseResponseCallback.java | 29 +++++ .../MergedBlockMetaResponseCallback.java | 35 ++++++ .../network/client/RpcResponseCallback.java | 5 +- .../spark/network/client/TransportClient.java | 25 +++- .../client/TransportResponseHandler.java | 28 ++++- .../spark/network/crypto/AuthRpcHandler.java | 5 + .../protocol/MergedBlockMetaRequest.java} | 77 +++++++------ .../protocol/MergedBlockMetaSuccess.java | 92 +++++++++++++++ .../spark/network/protocol/Message.java | 5 +- .../network/protocol/MessageDecoder.java | 6 + .../server/AbstractAuthRpcHandler.java | 5 + .../spark/network/server/RpcHandler.java | 18 +++ .../server/TransportRequestHandler.java | 29 +++++ .../network/TransportRequestHandlerSuite.java | 60 ++++++++++ .../TransportResponseHandlerSuite.java | 39 +++++++ .../protocol/MergedBlockMetaSuccessSuite.java | 106 +++++++++++++++++ .../spark/network/sasl/SparkSaslSuite.java | 6 +- .../network/shuffle/BlockStoreClient.java | 8 +- .../network/shuffle/ExternalBlockHandler.java | 60 +++++----- .../shuffle/ExternalBlockStoreClient.java | 27 ++--- .../network/shuffle/MergedBlockMeta.java | 64 +++++++++++ .../shuffle/MergedBlocksMetaListener.java | 10 +- .../shuffle/MergedShuffleFileManager.java | 6 +- .../shuffle/OneForOneBlockFetcher.java | 22 +--- .../shuffle/RemoteBlockPushResolver.java | 108 +++++++++++++++--- .../protocol/BlockTransferMessage.java | 7 +- .../shuffle/protocol/MergedBlocksMeta.java | 75 ------------ .../shuffle/ExternalBlockHandlerSuite.java | 49 ++++---- .../shuffle/OneForOneBlockFetcherSuite.java | 20 ++++ .../shuffle/RemoteBlockPushResolverSuite.java | 33 +++--- 30 files changed, 808 insertions(+), 251 deletions(-) create mode 100644 common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java create mode 100644 common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java rename common/{network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java => network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java} (51%) create mode 100644 common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java create mode 100644 common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java new file mode 100644 index 0000000000000..bb086b3c9a88b --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java @@ -0,0 +1,29 @@ +/* + * 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.client; + +/** + * A basic callback. This is extended by {@link RpcResponseCallback} and + * {@link MergedBlockMetaResponseCallback} so that both RpcRequests and MergedBlockMetaRequests + * can be handled in {@link TransportResponseHandler} a similar way. + */ +public interface BaseResponseCallback { + + /** Exception either propagated from server or raised on client side. */ + void onFailure(Throwable e); +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java new file mode 100644 index 0000000000000..a0c8edf05c453 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java @@ -0,0 +1,35 @@ +/* + * 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.client; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * Callback for the result of a single + * {@link org.apache.spark.network.protocol.MergedBlockMetaRequest}. + */ +public interface MergedBlockMetaResponseCallback extends BaseResponseCallback { + /** + * Called upon receipt of a particular merged block meta. + * + * The given buffer will initially have a refcount of 1, but will be release()'d as soon as this + * call returns. You must therefore either retain() the buffer or copy its contents before + * returning. + */ + void onSuccess(int numChunks, ManagedBuffer buffer); +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java index 6afc63f71bb3d..a3b8cb1d90a2e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java @@ -23,7 +23,7 @@ * Callback for the result of a single RPC. This will be invoked once with either success or * failure. */ -public interface RpcResponseCallback { +public interface RpcResponseCallback extends BaseResponseCallback { /** * Successful serialized result from server. * @@ -31,7 +31,4 @@ public interface RpcResponseCallback { * Please copy the content of `response` if you want to use it after `onSuccess` returns. */ void onSuccess(ByteBuffer response); - - /** Exception either propagated from server or raised on client side. */ - void onFailure(Throwable e); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 6dcc703e92669..589810fdc578c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -200,6 +200,27 @@ public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { return requestId; } + /** + * Sends a FetchMergedBlockMeta message to the server-side. The response of this message is + * either a {@link MergedBlockMetaSuccess} or {@link RpcFailure}. + * + * @param appId applicationId. + * @param mergedBlockId merged block Id. + * @param callback callback the handle the reply. + */ + public void sendMergedBlockMetaReq(String appId, String mergedBlockId, + MergedBlockMetaResponseCallback callback) { + long requestId = requestId(); + if (logger.isTraceEnabled()) { + logger.trace( + "Sending RPC {} to fetch merged block meta to {}", requestId, getRemoteAddress(channel)); + } + handler.addRpcRequest(requestId, callback); + RpcChannelListener listener = new RpcChannelListener(requestId, callback); + channel.writeAndFlush( + new MergedBlockMetaRequest(requestId, appId, mergedBlockId)).addListener(listener); + } + /** * Send data to the remote end as a stream. This differs from stream() in that this is a request * to *send* data to the remote end, not to receive it from the remote. @@ -349,9 +370,9 @@ void handleFailure(String errorMsg, Throwable cause) throws Exception {} private class RpcChannelListener extends StdChannelListener { final long rpcRequestId; - final RpcResponseCallback callback; + final BaseResponseCallback callback; - RpcChannelListener(long rpcRequestId, RpcResponseCallback callback) { + RpcChannelListener(long rpcRequestId, BaseResponseCallback callback) { super("RPC " + rpcRequestId); this.rpcRequestId = rpcRequestId; this.callback = callback; diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 2f143f77fa4ae..1ee06a3756e49 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -33,6 +33,7 @@ import org.apache.spark.network.protocol.ChunkFetchFailure; import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.MergedBlockMetaSuccess; import org.apache.spark.network.protocol.ResponseMessage; import org.apache.spark.network.protocol.RpcFailure; import org.apache.spark.network.protocol.RpcResponse; @@ -56,7 +57,7 @@ public class TransportResponseHandler extends MessageHandler { private final Map outstandingFetches; - private final Map outstandingRpcs; + private final Map outstandingRpcs; private final Queue> streamCallbacks; private volatile boolean streamActive; @@ -81,7 +82,7 @@ public void removeFetchRequest(StreamChunkId streamChunkId) { outstandingFetches.remove(streamChunkId); } - public void addRpcRequest(long requestId, RpcResponseCallback callback) { + public void addRpcRequest(long requestId, BaseResponseCallback callback) { updateTimeOfLastRequest(); outstandingRpcs.put(requestId, callback); } @@ -112,7 +113,7 @@ private void failOutstandingRequests(Throwable cause) { logger.warn("ChunkReceivedCallback.onFailure throws exception", e); } } - for (Map.Entry entry : outstandingRpcs.entrySet()) { + for (Map.Entry entry : outstandingRpcs.entrySet()) { try { entry.getValue().onFailure(cause); } catch (Exception e) { @@ -184,7 +185,7 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof RpcResponse) { RpcResponse resp = (RpcResponse) message; - RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + RpcResponseCallback listener = (RpcResponseCallback) outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); @@ -198,7 +199,7 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof RpcFailure) { RpcFailure resp = (RpcFailure) message; - RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + BaseResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.errorString); @@ -206,6 +207,23 @@ public void handle(ResponseMessage message) throws Exception { outstandingRpcs.remove(resp.requestId); listener.onFailure(new RuntimeException(resp.errorString)); } + } else if (message instanceof MergedBlockMetaSuccess) { + MergedBlockMetaSuccess resp = (MergedBlockMetaSuccess) message; + MergedBlockMetaResponseCallback listener = + (MergedBlockMetaResponseCallback) outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn( + "Ignoring response for MergedBlockMetaRequest {} from {} ({} bytes) since it is not" + + " outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); + resp.body().release(); + } else { + outstandingRpcs.remove(resp.requestId); + try { + listener.onSuccess(resp.getNumChunks(), resp.body()); + } finally { + resp.body().release(); + } + } } else if (message instanceof StreamResponse) { StreamResponse resp = (StreamResponse) message; Pair entry = streamCallbacks.poll(); diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java index dd31c955350f1..8f0a40c380219 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -138,4 +138,9 @@ protected boolean doAuthChallenge( LOG.debug("Authorization successful for client {}.", channel.remoteAddress()); return true; } + + @Override + public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { + return saslHandler.getMergedBlockMetaReqHandler(); + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java similarity index 51% rename from common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java index 863aa1bfae774..bf32e6134b61d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchMergedBlocksMeta.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java @@ -15,69 +15,72 @@ * limitations under the License. */ -package org.apache.spark.network.shuffle.protocol; - -import java.util.Arrays; +package org.apache.spark.network.protocol; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; -import org.apache.spark.network.protocol.Encoders; - /** - * Request to find the meta information for the specified merged blocks. The meta information - * currently contains only the number of chunks in each merged blocks. + * Request to find the meta information for the specified merged block. The meta information + * contains the number of chunks in the merged blocks and the maps ids in each chunk. */ -public class FetchMergedBlocksMeta extends BlockTransferMessage { +public class MergedBlockMetaRequest extends AbstractMessage implements RequestMessage { + public final long requestId; public final String appId; - public final String[] blockIds; + public final String blockId; - public FetchMergedBlocksMeta(String appId, String[] blockIds) { + public MergedBlockMetaRequest(long requestId, String appId, String blockId) { + super(null, false); + this.requestId = requestId; this.appId = appId; - this.blockIds = blockIds; + this.blockId = blockId; } @Override - protected Type type() { return Type.FETCH_MERGED_BLOCKS_META; } + public Type type() { + return Type.MergedBlockMetaRequest; + } @Override - public int hashCode() { - return appId.hashCode() * 41 + Arrays.hashCode(blockIds); + public int encodedLength() { + return 8 + Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(blockId); } @Override - public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("blockIds", Arrays.toString(blockIds)) - .toString(); + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, blockId); } - @Override - public boolean equals(Object other) { - if (other instanceof FetchMergedBlocksMeta) { - FetchMergedBlocksMeta o = (FetchMergedBlocksMeta) other; - return Objects.equal(appId, o.appId) - && Arrays.equals(blockIds, o.blockIds); - } - return false; + public static MergedBlockMetaRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + String appId = Encoders.Strings.decode(buf); + String blockId = Encoders.Strings.decode(buf); + return new MergedBlockMetaRequest(requestId, appId, blockId); } @Override - public int encodedLength() { - return Encoders.Strings.encodedLength(appId) - + Encoders.StringArrays.encodedLength(blockIds); + public int hashCode() { + return Objects.hashCode(requestId, appId, blockId); } @Override - public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - Encoders.StringArrays.encode(buf, blockIds); + public boolean equals(Object other) { + if (other instanceof MergedBlockMetaRequest) { + MergedBlockMetaRequest o = (MergedBlockMetaRequest) other; + return requestId == o.requestId && Objects.equal(appId, o.appId) && Objects.equal(blockId, + o.blockId); + } + return false; } - public static FetchMergedBlocksMeta decode(ByteBuf buf) { - String appId = Encoders.Strings.decode(buf); - String[] blockIds = Encoders.StringArrays.decode(buf); - return new FetchMergedBlocksMeta(appId, blockIds); + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("appId", appId) + .add("blockId", blockId) + .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java new file mode 100644 index 0000000000000..a89e9d3103a47 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java @@ -0,0 +1,92 @@ +/* + * 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.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * Response to {@link MergedBlockMetaRequest} request. + * Note that the server-side encoding of this messages does NOT include the buffer itself. + */ +public class MergedBlockMetaSuccess extends AbstractResponseMessage { + public final long requestId; + public final int numChunks; + + public MergedBlockMetaSuccess( + long requestId, int numChunks, ManagedBuffer chunkBitmapsBuffer) { + super(chunkBitmapsBuffer, true); + this.requestId = requestId; + this.numChunks = numChunks; + } + + @Override + public Type type() { + return Type.MergedBlockMetaSuccess; + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, numChunks); + } + + @Override + public String toString() { + return com.google.common.base.Objects.toStringHelper(this) + .add("requestId", requestId) + .add("numChunks", numChunks) + .toString(); + } + + @Override + public int encodedLength() { + return 8 + 4; + } + + /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + buf.writeInt(numChunks); + } + + public int getNumChunks() { + return numChunks; + } + + public ManagedBuffer getChunkBitmapsBuffer() { + return body(); + } + + /** Decoding uses the given ByteBuf as our data, and will retain() it. */ + public static MergedBlockMetaSuccess decode(ByteBuf buf) { + long requestId = buf.readLong(); + int numChunks = buf.readInt(); + buf.retain(); + NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate()); + return new MergedBlockMetaSuccess(requestId, numChunks, managedBuf); + } + + @Override + public ResponseMessage createFailureResponse(String error) { + return new RpcFailure(requestId, error); + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java index 0ccd70c03aba8..4a2643256fcb4 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -37,7 +37,8 @@ enum Type implements Encodable { ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), RpcRequest(3), RpcResponse(4), RpcFailure(5), StreamRequest(6), StreamResponse(7), StreamFailure(8), - OneWayMessage(9), UploadStream(10), User(-1); + OneWayMessage(9), UploadStream(10), User(-1), + MergedBlockMetaRequest(11), MergedBlockMetaSuccess(12); private final byte id; @@ -66,6 +67,8 @@ public static Type decode(ByteBuf buf) { case 8: return StreamFailure; case 9: return OneWayMessage; case 10: return UploadStream; + case 11: return MergedBlockMetaRequest; + case 12: return MergedBlockMetaSuccess; case -1: throw new IllegalArgumentException("User type messages cannot be decoded."); default: throw new IllegalArgumentException("Unknown message type: " + id); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java index bf80aed0afe10..98f7f612a486b 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -83,6 +83,12 @@ private Message decode(Message.Type msgType, ByteBuf in) { case UploadStream: return UploadStream.decode(in); + case MergedBlockMetaRequest: + return MergedBlockMetaRequest.decode(in); + + case MergedBlockMetaSuccess: + return MergedBlockMetaSuccess.decode(in); + default: throw new IllegalArgumentException("Unexpected message type: " + msgType); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java index 92eb886283448..95fde677624ff 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java @@ -104,4 +104,9 @@ public void exceptionCaught(Throwable cause, TransportClient client) { public boolean isAuthenticated() { return isAuthenticated; } + + @Override + public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { + return delegate.getMergedBlockMetaReqHandler(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java index 38569baf82bce..8a46d37ae25e7 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -22,9 +22,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.StreamCallbackWithID; import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.MergedBlockMetaRequest; /** * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. @@ -32,6 +34,10 @@ public abstract class RpcHandler { private static final RpcResponseCallback ONE_WAY_CALLBACK = new OneWayRpcCallback(); + private static final MergedBlockMetaReqHandler NOOP_MERGED_BLOCK_META_REQ_HANDLER = + (client, appId, mergedBlockId, callback) -> { + // do nothing + }; /** * Receive a single RPC message. Any exception thrown while in this method will be sent back to @@ -100,6 +106,10 @@ public void receive(TransportClient client, ByteBuffer message) { receive(client, message, ONE_WAY_CALLBACK); } + public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { + return NOOP_MERGED_BLOCK_META_REQ_HANDLER; + } + /** * Invoked when the channel associated with the given client is active. */ @@ -129,4 +139,12 @@ public void onFailure(Throwable e) { } + /** + * Handler for {@link MergedBlockMetaRequest}. + */ + public interface MergedBlockMetaReqHandler { + void receiveMergeBlockMetaReq( + TransportClient client, String appId, String mergedBlockId, + MergedBlockMetaResponseCallback callback); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 67e4140baee89..a020dd600c6a7 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -114,6 +114,8 @@ public void handle(RequestMessage request) throws Exception { processStreamRequest((StreamRequest) request); } else if (request instanceof UploadStream) { processStreamUpload((UploadStream) request); + } else if (request instanceof MergedBlockMetaRequest) { + processMergedBlockMetaRequest((MergedBlockMetaRequest) request); } else { throw new IllegalArgumentException("Unknown request type: " + request); } @@ -294,6 +296,33 @@ private void processOneWayMessage(OneWayMessage req) { } } + private void processMergedBlockMetaRequest(final MergedBlockMetaRequest req) { + try { + rpcHandler.getMergedBlockMetaReqHandler() + .receiveMergeBlockMetaReq(reverseClient, req.appId, req.blockId, + new MergedBlockMetaResponseCallback() { + + @Override + public void onSuccess(int numChunks, ManagedBuffer buffer) { + logger.trace("Sending response for {}: app {} merged blockId {} numChunks {}", + req.requestId, req.appId, req.blockId, numChunks); + respond(new MergedBlockMetaSuccess(req.requestId, numChunks, buffer)); + } + + @Override + public void onFailure(Throwable e) { + logger.trace("Failed to send response for {}: app {} merged blockId {}", + req.requestId, req.appId, req.blockId); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }); + } catch (Exception e) { + logger.error("Error while invoking receiveMergeBlockMetaReq() for app {} merged block id {} ", + req.appId, req.blockId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + } + /** * Make a full copy of a nio ByteBuffer. */ 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 0a6447176237a..74dcd4ee52ed7 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 @@ -17,6 +17,7 @@ package org.apache.spark.network; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -24,16 +25,19 @@ import org.junit.Assert; import org.junit.Test; +import static org.junit.Assert.*; import static org.mockito.Mockito.*; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.protocol.*; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportRequestHandler; public class TransportRequestHandlerSuite { @@ -109,4 +113,60 @@ public void handleStreamRequest() throws Exception { streamManager.connectionTerminated(channel); Assert.assertEquals(0, streamManager.numStreamStates()); } + + @Test + public void handleMergedBlockMetaRequest() throws Exception { + + RpcHandler.MergedBlockMetaReqHandler metaHandler = (client, appId, mergedBlockId, callback) -> { + if (!mergedBlockId.isEmpty()) { + callback.onSuccess(2, mock(ManagedBuffer.class)); + } else { + callback.onFailure(new RuntimeException("empty block")); + } + }; + + RpcHandler rpcHandler = new RpcHandler() { + @Override + public void receive( + TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + } + + @Override + public StreamManager getStreamManager() { + return null; + } + + @Override + public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { + return metaHandler; + } + }; + + Channel channel = mock(Channel.class); + List> responseAndPromisePairs = + new ArrayList<>(); + when(channel.writeAndFlush(any())) + .thenAnswer(invocationOnMock0 -> { + Object response = invocationOnMock0.getArguments()[0]; + ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel); + responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture)); + return channelFuture; + }); + + TransportClient reverseClient = mock(TransportClient.class); + TransportRequestHandler requestHandler = + new TransportRequestHandler(channel, reverseClient, rpcHandler, 2L, null); + + MergedBlockMetaRequest validMetaReq = new MergedBlockMetaRequest(19, "app1", "mergedBlock"); + requestHandler.handle(validMetaReq); + assertEquals(1, responseAndPromisePairs.size()); + assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof MergedBlockMetaSuccess); + assertEquals(2, + ((MergedBlockMetaSuccess) (responseAndPromisePairs.get(0).getLeft())).getNumChunks()); + + MergedBlockMetaRequest invalidMetaReq = new MergedBlockMetaRequest(21, "app1", ""); + requestHandler.handle(invalidMetaReq); + assertEquals(2, responseAndPromisePairs.size()); + assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof RpcFailure); + } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java index b4032c4c3f031..4bcb8bd03951c 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -23,17 +23,20 @@ import io.netty.channel.Channel; import io.netty.channel.local.LocalChannel; import org.junit.Test; +import org.mockito.ArgumentCaptor; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.StreamCallback; import org.apache.spark.network.client.TransportResponseHandler; import org.apache.spark.network.protocol.ChunkFetchFailure; import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.MergedBlockMetaSuccess; import org.apache.spark.network.protocol.RpcFailure; import org.apache.spark.network.protocol.RpcResponse; import org.apache.spark.network.protocol.StreamChunkId; @@ -167,4 +170,40 @@ public void failOutstandingStreamCallbackOnException() throws Exception { verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); } + + @Test + public void handleSuccessfulMergedBlockMeta() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); + handler.addRpcRequest(13, callback); + assertEquals(1, handler.numOutstandingRequests()); + + // This response should be ignored. + handler.handle(new MergedBlockMetaSuccess(22, 2, + new NioManagedBuffer(ByteBuffer.allocate(7)))); + assertEquals(1, handler.numOutstandingRequests()); + + ByteBuffer resp = ByteBuffer.allocate(10); + handler.handle(new MergedBlockMetaSuccess(13, 2, new NioManagedBuffer(resp))); + ArgumentCaptor bufferCaptor = ArgumentCaptor.forClass(NioManagedBuffer.class); + verify(callback, times(1)).onSuccess(eq(2), bufferCaptor.capture()); + assertEquals(resp, bufferCaptor.getValue().nioByteBuffer()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedMergedBlockMeta() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); + handler.addRpcRequest(51, callback); + assertEquals(1, handler.numOutstandingRequests()); + + // This response should be ignored. + handler.handle(new RpcFailure(6, "failed")); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(51, "failed")); + verify(callback, times(1)).onFailure(any()); + assertEquals(0, handler.numOutstandingRequests()); + } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java new file mode 100644 index 0000000000000..b6dd371044690 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java @@ -0,0 +1,106 @@ +/* + * 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.protocol; + +import java.io.File; +import java.io.FileOutputStream; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.util.List; + +import com.google.common.collect.Lists; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import org.junit.Assert; +import org.junit.Test; +import org.roaringbitmap.RoaringBitmap; + +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.util.ByteArrayWritableChannel; +import org.apache.spark.network.util.TransportConf; + +/** + * Test for {@link MergedBlockMetaSuccess}. + */ +public class MergedBlockMetaSuccessSuite { + + @Test + public void testMergedBlocksMetaEncodeDecode() throws Exception { + File chunkMetaFile = new File("target/mergedBlockMetaTest"); + Files.deleteIfExists(chunkMetaFile.toPath()); + RoaringBitmap chunk1 = new RoaringBitmap(); + chunk1.add(1); + chunk1.add(3); + RoaringBitmap chunk2 = new RoaringBitmap(); + chunk2.add(2); + chunk2.add(4); + RoaringBitmap[] expectedChunks = new RoaringBitmap[]{chunk1, chunk2}; + FileChannel fileChannel = new FileOutputStream(chunkMetaFile, true).getChannel(); + ByteBuf buf = Unpooled.buffer(); + for (int i = 0; i < expectedChunks.length; i++) { + Encoders.Bitmaps.encode(buf, expectedChunks[i]); + } + fileChannel.write(buf.nioBuffer()); + fileChannel.close(); + buf.release(); + + TransportConf conf = mock(TransportConf.class); + when(conf.lazyFileDescriptor()).thenReturn(false); + long requestId = 1L; + MergedBlockMetaSuccess expectedMeta = new MergedBlockMetaSuccess(requestId, 2, + new FileSegmentManagedBuffer(conf, chunkMetaFile, 0, chunkMetaFile.length())); + + List out = Lists.newArrayList(); + ChannelHandlerContext context = mock(ChannelHandlerContext.class); + when(context.alloc()).thenReturn(ByteBufAllocator.DEFAULT); + + MessageEncoder.INSTANCE.encode(context, expectedMeta, out); + Assert.assertEquals(1, out.size()); + MessageWithHeader msgWithHeader = (MessageWithHeader) out.remove(0); + + ByteArrayWritableChannel writableChannel = + new ByteArrayWritableChannel((int) msgWithHeader.count()); + while (msgWithHeader.transfered() < msgWithHeader.count()) { + msgWithHeader.transferTo(writableChannel, msgWithHeader.transfered()); + } + ByteBuf messageBuf = Unpooled.wrappedBuffer(writableChannel.getData()); + messageBuf.readLong(); // frame length + MessageDecoder.INSTANCE.decode(mock(ChannelHandlerContext.class), messageBuf, out); + Assert.assertEquals(1, out.size()); + MergedBlockMetaSuccess decoded = (MergedBlockMetaSuccess) out.get(0); + Assert.assertEquals("merged block", expectedMeta.requestId, decoded.requestId); + Assert.assertEquals("num chunks", expectedMeta.getNumChunks(), decoded.getNumChunks()); + + ByteBuf responseBuf = Unpooled.wrappedBuffer(decoded.body().nioByteBuffer()); + RoaringBitmap[] responseBitmaps = new RoaringBitmap[expectedMeta.getNumChunks()]; + for (int i = 0; i < expectedMeta.getNumChunks(); i++) { + responseBitmaps[i] = Encoders.Bitmaps.decode(responseBuf); + } + + Assert.assertEquals( + "num of roaring bitmaps", expectedMeta.getNumChunks(), responseBitmaps.length); + for (int i = 0; i < expectedMeta.getNumChunks(); i++) { + Assert.assertEquals("chunk bitmap " + i, expectedChunks[i], responseBitmaps[i]); + } + Files.delete(chunkMetaFile.toPath()); + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index ecaeec98da182..cc8b8bcc88e9a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -357,8 +357,10 @@ public void testRpcHandlerDelegate() throws Exception { public void testDelegates() throws Exception { Method[] rpcHandlerMethods = RpcHandler.class.getDeclaredMethods(); for (Method m : rpcHandlerMethods) { - Method delegate = SaslRpcHandler.class.getMethod(m.getName(), m.getParameterTypes()); - assertNotEquals(delegate.getDeclaringClass(), RpcHandler.class); + if (!m.getName().contains("lambda")) { + Method delegate = SaslRpcHandler.class.getMethod(m.getName(), m.getParameterTypes()); + assertNotEquals(delegate.getDeclaringClass(), RpcHandler.class); + } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index d2d7fbf8b3145..13b2779aa79e2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -157,17 +157,17 @@ public void pushBlocks( } /** - * Get the meta information of the merged blocks from the remote shuffle service. + * Get the meta information of a merged block from the remote shuffle service. * * @param host the host of the remote node. * @param port the port of the remote node. - * @param blocks block ids of the merged shuffle blocks. + * @param block block id of the merged shuffle block. * @param listener the listener to receive chunk counts. */ - public void getMergedBlocksMeta( + public void getMergedBlockMeta( String host, int port, - String[] blocks, + String block, MergedBlocksMetaListener listener) { throw new UnsupportedOperationException(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 0aee4b3d86dd7..55a1f42fe6ce2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -32,6 +32,7 @@ import com.codahale.metrics.Timer; import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; +import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.StreamCallbackWithID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +56,8 @@ * Blocks are registered with the "one-for-one" strategy, meaning each Transport-layer Chunk * is equivalent to one block. */ -public class ExternalBlockHandler extends RpcHandler { +public class ExternalBlockHandler extends RpcHandler + implements RpcHandler.MergedBlockMetaReqHandler { private static final Logger logger = LoggerFactory.getLogger(ExternalBlockHandler.class); private static final String SHUFFLE_BLOCK_PREFIX = "shuffle"; private static final String SHUFFLE_CHUNK_PREFIX = "shuffleChunk"; @@ -166,29 +168,6 @@ protected void handleMessage( } finally { responseDelayContext.stop(); } - } else if (msgObj instanceof FetchMergedBlocksMeta) { - final Timer.Context responseDelayContext = - metrics.fetchMergedBlocksMetaLatencyMillis.time(); - try { - FetchMergedBlocksMeta mergedMetaReq = (FetchMergedBlocksMeta) msgObj; - checkAuth(client, mergedMetaReq.appId); - int[] chunkCounts = new int[mergedMetaReq.blockIds.length]; - for (int i = 0; i < mergedMetaReq.blockIds.length; i++) { - String[] blockIdParts = mergedMetaReq.blockIds[i].split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) { - throw new IllegalArgumentException( - "Unexpected shuffle block id format: " + mergedMetaReq.blockIds[i]); - } - chunkCounts[i] = - mergeManager.getChunkCount(mergedMetaReq.appId, Integer.parseInt(blockIdParts[1]), - Integer.parseInt(blockIdParts[3])); - logger.debug( - "Merged block {} with {} chunks", mergedMetaReq.blockIds[i], chunkCounts[i]); - } - callback.onSuccess(new MergedBlocksMeta(chunkCounts).toByteBuffer()); - } finally { - responseDelayContext.stop(); - } } else if (msgObj instanceof RegisterExecutor) { final Timer.Context responseDelayContext = metrics.registerExecutorRequestLatencyMillis.time(); @@ -232,6 +211,35 @@ protected void handleMessage( } } + @Override + public void receiveMergeBlockMetaReq( + TransportClient client, String appId, String mergedBlockId, + MergedBlockMetaResponseCallback callback) { + + final Timer.Context responseDelayContext = metrics.fetchMergedBlocksMetaLatencyMillis.time(); + try { + checkAuth(client, appId); + String[] blockIdParts = mergedBlockId.split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) { + throw new IllegalArgumentException( + "Unexpected shuffle block id format: " + mergedBlockId); + } + MergedBlockMeta mergedMeta = + mergeManager.getMergedBlockMeta(appId, Integer.parseInt(blockIdParts[1]), + Integer.parseInt(blockIdParts[3])); + logger.debug( + "Merged block chunks {} : {} ", mergedBlockId, mergedMeta.getNumChunks()); + callback.onSuccess(mergedMeta.getNumChunks(), mergedMeta.getChunksBitmapBuffer()); + } finally { + responseDelayContext.stop(); + } + } + + @Override + public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { + return this; + } + @Override public void exceptionCaught(Throwable cause, TransportClient client) { metrics.caughtExceptions.inc(); @@ -547,8 +555,8 @@ public ManagedBuffer getMergedBlockData( } @Override - public int getChunkCount(String appId, int shuffleId, int reduceId) { - throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { + return null; } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index a5491a4358ced..cbde0d48bbad8 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; @@ -161,35 +162,31 @@ public void pushBlocks( } @Override - public void getMergedBlocksMeta( + public void getMergedBlockMeta( String host, int port, - String[] blocks, + String mergedBlockId, MergedBlocksMetaListener listener) { checkInit(); - logger.debug("Get merged blocks meta from {}:{} with {} blocks", host, port, blocks.length); + logger.debug("Get merged blocks meta from {}:{} for block {}", host, port, mergedBlockId); try { TransportClient client = clientFactory.createClient(host, port); - FetchMergedBlocksMeta mergedBlocksMetaReq = new FetchMergedBlocksMeta(appId, blocks); - client.sendRpc(mergedBlocksMetaReq.toByteBuffer(), new RpcResponseCallback() { + client.sendMergedBlockMetaReq(appId, mergedBlockId, new MergedBlockMetaResponseCallback() { @Override - public void onSuccess(ByteBuffer response) { - - MergedBlocksMeta blocksMeta = - (MergedBlocksMeta) BlockTransferMessage.Decoder.fromByteBuffer(response); - logger.trace("Successfully got merged blocks meta {}", blocksMeta); - listener.onSuccess(blocks, blocksMeta.numChunks); + public void onSuccess(int numChunks, ManagedBuffer buffer) { + logger.trace("Successfully got merged block meta for {}", mergedBlockId); + listener.onSuccess(mergedBlockId, new MergedBlockMeta(numChunks, buffer)); } @Override public void onFailure(Throwable e) { - logger.error("Failed while getting merged blocks meta", e); - listener.onFailure(blocks, e); + logger.error("Failed while getting merged block meta", e); + listener.onFailure(mergedBlockId, e); } }); } catch (Exception e) { - logger.error("Exception while getting merged blocks meta", e); - listener.onFailure(blocks, e); + logger.error("Exception while getting merged block meta", e); + listener.onFailure(mergedBlockId, e); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java new file mode 100644 index 0000000000000..e9d9e53495469 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java @@ -0,0 +1,64 @@ +/* + * 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; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.roaringbitmap.RoaringBitmap; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.protocol.Encoders; + +/** + * Contains meta information for a merged block. Currently this information constitutes: + * 1. Number of chunks in a merged shuffle block. + * 2. Bitmaps for each chunk in the merged block. A chunk bitmap contains all the mapIds that were + * merged to that merged block chunk. + */ +public class MergedBlockMeta { + private final int numChunks; + private final ManagedBuffer chunksBitmapBuffer; + + public MergedBlockMeta(int numChunks, ManagedBuffer chunksBitmapBuffer) { + this.numChunks = numChunks; + this.chunksBitmapBuffer = Preconditions.checkNotNull(chunksBitmapBuffer); + } + + public int getNumChunks() { + return numChunks; + } + + public ManagedBuffer getChunksBitmapBuffer() { + return chunksBitmapBuffer; + } + + public RoaringBitmap[] readChunkBitmaps() throws IOException { + ByteBuf buf = Unpooled.wrappedBuffer(chunksBitmapBuffer.nioByteBuffer()); + List bitmaps = new ArrayList<>(); + while(buf.isReadable()) { + bitmaps.add(Encoders.Bitmaps.decode(buf)); + } + assert (bitmaps.size() == numChunks); + return bitmaps.toArray(new RoaringBitmap[0]); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java index 3ca66e06aad51..55ea9bf81a24e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java @@ -27,15 +27,15 @@ public interface MergedBlocksMetaListener extends EventListener { /** * Called after successfully receiving the meta of merged blocks. Currently, the meta only * includes the count of chunks in a merged block. - * @param blocks merged block Ids. - * @param numChunks number of chunks per merged block. + * @param mergedBlockId merged block Id. + * @param meta contains meta information of a merged block. */ - void onSuccess(String[] blocks, int[] numChunks); + void onSuccess(String mergedBlockId, MergedBlockMeta meta); /** * Called when there is an exception while fetching the meta of merged blocks. - * @param blocks merged block Ids. + * @param mergedBlockId merged block Id. * @param exception exception getting chunk counts. */ - void onFailure(String[] blocks, Throwable exception); + void onFailure(String mergedBlockId, Throwable exception); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java index 14044e49b454b..73dc2e207178c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -92,12 +92,12 @@ public interface MergedShuffleFileManager { ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId); /** - * Get the number of chunks for a given merged shuffle file. + * Get the meta information of a merged block. * * @param appId application ID * @param shuffleId shuffle ID * @param reduceId reducer ID - * @return number of chunks + * @return meta information of a merged block */ - int getChunkCount(String appId, int shuffleId, int reduceId); + MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 07b81aa3f9545..c22622b14ae31 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -222,8 +222,8 @@ public void onFailure(int chunkIndex, Throwable e) { // If failed block is a merged block, we only fail this block and do not // fail the remaining blocks. The failed merged block will be retried by // falling back to fetching the original unmerged blocks. - if (isMergedBlock(blockIds[chunkIndex])) { - failSingleMergedBlock(blockIds[chunkIndex], e); + if ((blockIds[chunkIndex]).startsWith(SHUFFLE_CHUNK_PREFIX)) { + failSingleBlockChunk(blockIds[chunkIndex], e); } else { // On receipt of a failure, fail every block from chunkIndex onwards. String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); @@ -280,23 +280,14 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { } } - private void failSingleMergedBlock(String mergedBlockId, Throwable e) { + private void failSingleBlockChunk(String shuffleBlockChunkId, Throwable e) { try { - listener.onBlockFetchFailure(mergedBlockId, e); + listener.onBlockFetchFailure(shuffleBlockChunkId, e); } catch (Exception e2) { logger.error("Error in block fetch failure callback", e2); } } - /** Verify if a given block id represents a merged block. */ - // TODO this needs to be updated to support merged shuffle chunks. Right now - // TODO fallback handling for merged shuffle chunk is not enabled. - private boolean isMergedBlock(String blockId) { - String[] blockIdParts = blockId.split("_"); - return blockIdParts.length == 4 && blockIdParts[0].equals("shuffle") - && blockIdParts[2].equals("-1"); - } - private class DownloadCallback implements StreamCallback { private DownloadFileWritableChannel channel = null; @@ -327,9 +318,8 @@ public void onComplete(String streamId) throws IOException { @Override public void onFailure(String streamId, Throwable cause) throws IOException { channel.close(); - // Do not fail the remaining blocks if the failed block is a merged block. - if (isMergedBlock(blockIds[chunkIndex])) { - failSingleMergedBlock(blockIds[chunkIndex], cause); + if ((blockIds[chunkIndex]).startsWith(SHUFFLE_CHUNK_PREFIX)) { + failSingleBlockChunk(blockIds[chunkIndex], cause); } else { // On receipt of a failure, fail every block from chunkIndex onwards. String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 7f5e8aafcd42d..df13546a17456 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -47,6 +47,8 @@ import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +56,7 @@ import org.apache.spark.network.buffer.FileSegmentManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.StreamCallbackWithID; +import org.apache.spark.network.protocol.Encoders; import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; import org.apache.spark.network.shuffle.protocol.MergeStatuses; import org.apache.spark.network.shuffle.protocol.PushBlockStream; @@ -119,16 +122,18 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // two cases accordingly by checking if the file already exists. File mergedShuffleFile = getMergedShuffleFile(key); File mergedIndexFile = getMergedIndexFile(id); + File mergedMetaFile = getMergedMetaFile(id); try { if (mergedShuffleFile.exists()) { return null; } else { - return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile); + return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile, mergedMetaFile); } } catch (IOException e) { logger.error( - "Cannot create merged shuffle partition {} with shuffle file {} and index file {}", key, - mergedIndexFile.getAbsolutePath(), mergedIndexFile.getAbsolutePath()); + "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and " + + "meta file {}", key, mergedIndexFile.getAbsolutePath(), + mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath()); throw new RuntimeException(String.format( "Cannot initialize merged shuffle partition %s", key.toString()), e); } @@ -136,7 +141,10 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( } @Override - public int getChunkCount(String appId, int shuffleId, int reduceId) { + public MergedBlockMeta getMergedBlockMeta( + String appId, + int shuffleId, + int reduceId) { AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); File indexFile = getMergedIndexFile(id); if (!indexFile.exists()) { @@ -146,7 +154,18 @@ public int getChunkCount(String appId, int shuffleId, int reduceId) { } int size = (int) indexFile.length(); // First entry is the zero offset - return (size / Long.BYTES) - 1; + int numChunks = (size / Long.BYTES) - 1; + File metaFile = getMergedMetaFile(id); + if (!metaFile.exists()) { + throw new RuntimeException( + String.format("Application merged shuffle meta file is not found (id=%s)", + id.toString())); + } + FileSegmentManagedBuffer chunkBitMaps = + new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length()); + logger.trace( + "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks); + return new MergedBlockMeta(numChunks, chunkBitMaps); } @SuppressWarnings("UnstableApiUsage") @@ -201,6 +220,11 @@ private File getMergedIndexFile(AppShufflePartitionId id) { return getFile(id.appId, indexName); } + private File getMergedMetaFile(AppShufflePartitionId id) { + String metaName = id.generateMetaFileName(); + return getFile(id.appId, metaName); + } + @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); @@ -472,14 +496,19 @@ public void onComplete(String streamId) throws IOException { deferredBufs = null; } long updatedPos = partitionInfo.getPosition() + length; + boolean indexUpdated = false; if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { - partitionInfo.updateLastChunkOffset(updatedPos); + partitionInfo.updateLastChunkOffset(updatedPos, mapId); + indexUpdated = true; } partitionInfo.setPosition(updatedPos); partitionInfo.setCurrentMapId(-1); // update merged results partitionInfo.blockMerged(mapId); + if (indexUpdated) { + partitionInfo.resetChunkTracker(); + } } else { deferredBufs = null; canWrite = false; @@ -540,7 +569,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc try { partition.channel.truncate(partition.getPosition()); if (partition.getPosition() != partition.getLastChunkOffset()) { - partition.updateLastChunkOffset(partition.getPosition()); + partition.updateLastChunkOffset(partition.getPosition(), partition.lastMergedMapId); } bitmaps.add(partition.mapTracker); reduceIds.add(partitionId.reduceId); @@ -551,6 +580,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } finally { try { partition.channel.close(); + partition.metaChannel.close(); partition.indexWriteStream.close(); } catch (IOException closeEx) { logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId, @@ -626,6 +656,10 @@ String generateIndexFileName() { return String.format("mergedShuffle_%s_%d_%d.index", appId, shuffleId, reduceId); } + String generateMetaFileName() { + return String.format("mergedShuffle_%s_%d_%d.meta", appId, shuffleId, reduceId); + } + boolean compareAppShuffleId(String appId, int shuffleId) { return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId; } @@ -648,11 +682,17 @@ public static class AppShufflePartitionInfo { private RoaringBitmap mapTracker; // The merged shuffle index file private final FileChannel indexChannel; + private final FileChannel metaChannel; private final DataOutputStream indexWriteStream; // The offset for the last chunk tracked in the index file for this shuffle partition private long lastChunkOffset; + private int lastMergedMapId; + + // Bitmap tracking which mapper's blocks are in shuffle chunk + private RoaringBitmap chunkTracker; + ByteBuf trackerBuf = null; - AppShufflePartitionInfo(File targetFile, File indexFile) throws IOException { + AppShufflePartitionInfo(File targetFile, File indexFile, File metaFile) throws IOException { targetFile.createNewFile(); this.targetFile = targetFile; this.channel = new FileOutputStream(targetFile, true).getChannel(); @@ -660,12 +700,15 @@ public static class AppShufflePartitionInfo { FileOutputStream fos = new FileOutputStream(indexFile, true); indexChannel = fos.getChannel(); this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); + metaFile.createNewFile(); + metaChannel = new FileOutputStream(metaFile, true).getChannel(); + this.currentMapId = -1; // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() - updateLastChunkOffset(0L); + updateLastChunkOffset(0L, -1); this.position = 0; this.encounteredFailure = false; - this.currentMapId = -1; this.mapTracker = new RoaringBitmap(); + this.chunkTracker = new RoaringBitmap(); } public long getPosition() { @@ -698,19 +741,54 @@ long getLastChunkOffset() { void blockMerged(int mapId) { mapTracker.add(mapId); + chunkTracker.add(mapId); + lastMergedMapId = mapId; } - void updateLastChunkOffset(long lastChunkOffset) throws IOException { - long startPos = indexChannel.position(); + void resetChunkTracker() { + chunkTracker.clear(); + } + + void updateLastChunkOffset(long lastChunkOffset, int mapId) throws IOException { + long idxStartPos = -1; try { + // update the chunk tracker to meta file before index file + writeChunkTracker(mapId); + idxStartPos = indexChannel.position(); + logger.trace("{} updated index with offset {}", targetFile.getName(), lastChunkOffset); indexWriteStream.writeLong(lastChunkOffset); - } catch(IOException ioe) { - // reset the position to avoid corrupting index files during exception. - indexChannel.position(startPos); + } catch (IOException ioe) { + if (idxStartPos != -1) { + // reset the position to avoid corrupting index files during exception. + logger.warn("{} reset index to position {}", targetFile.getName(), idxStartPos); + indexChannel.position(idxStartPos); + } throw ioe; } this.lastChunkOffset = lastChunkOffset; } + + private void writeChunkTracker(int mapId) throws IOException { + if (mapId == -1) { + return; + } + chunkTracker.add(mapId); + if (trackerBuf == null) { + trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker)); + } + Encoders.Bitmaps.encode(trackerBuf, chunkTracker); + long metaStartPos = metaChannel.position(); + try { + logger.trace("{} write chunk to meta file", targetFile.getName()); + metaChannel.write(trackerBuf.nioBuffer()); + } catch (IOException ioe) { + logger.warn("{} reset position of meta file to {}", targetFile.getName(), metaStartPos); + metaChannel.position(metaStartPos); + throw ioe; + } finally { + trackerBuf.clear(); + } + } } /** 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 b4a7aed819444..a55a6cf7ed939 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 @@ -49,8 +49,7 @@ public enum Type { HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11), PUSH_BLOCK_STREAM(12), FINALIZE_SHUFFLE_MERGE(13), MERGE_STATUSES(14), - FETCH_MERGED_BLOCKS_META(15), MERGED_BLOCKS_META(16), - FETCH_SHUFFLE_BLOCK_CHUNKS(17); + FETCH_SHUFFLE_BLOCK_CHUNKS(15); private final byte id; @@ -84,9 +83,7 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 12: return PushBlockStream.decode(buf); case 13: return FinalizeShuffleMerge.decode(buf); case 14: return MergeStatuses.decode(buf); - case 15: return FetchMergedBlocksMeta.decode(buf); - case 16: return MergedBlocksMeta.decode(buf); - case 17: return FetchShuffleBlockChunks.decode(buf); + case 15: return FetchShuffleBlockChunks.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/MergedBlocksMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java deleted file mode 100644 index 94c3e616491f0..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergedBlocksMeta.java +++ /dev/null @@ -1,75 +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.protocol; - -import java.util.Arrays; -import javax.annotation.Nonnull; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.protocol.Encoders; - -/** - * Response of {@link FetchMergedBlocksMeta}. - */ -public class MergedBlocksMeta extends BlockTransferMessage { - - public final int[] numChunks; - - public MergedBlocksMeta(@Nonnull int[] numChunks) { - this.numChunks = numChunks; - } - - @Override - protected Type type() { return Type.MERGED_BLOCKS_META; } - - @Override - public int hashCode() { - return Arrays.hashCode(numChunks); - } - - @Override - public String toString() { - Objects.ToStringHelper helper = Objects.toStringHelper(this); - return helper.add("numChunks", Arrays.toString(numChunks)).toString(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof MergedBlocksMeta) { - MergedBlocksMeta o = (MergedBlocksMeta) other; - return Arrays.equals(numChunks, o.numChunks); - } - return false; - } - - @Override - public int encodedLength() { - return Encoders.IntArrays.encodedLength(numChunks); - } - - @Override - public void encode(ByteBuf buf) { - Encoders.IntArrays.encode(buf, numChunks); - } - - public static MergedBlocksMeta decode(ByteBuf buf) { - return new MergedBlocksMeta(Encoders.IntArrays.decode(buf)); - } -} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index ddbddba4db0cb..909ff1ef6bf4d 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -23,6 +23,7 @@ import com.codahale.metrics.Meter; import com.codahale.metrics.Timer; +import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -40,11 +41,9 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.FetchMergedBlocksMeta; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; -import org.apache.spark.network.shuffle.protocol.MergedBlocksMeta; import org.apache.spark.network.shuffle.protocol.MergeStatuses; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.RegisterExecutor; @@ -234,28 +233,33 @@ public void testBadMessages() { @Test public void testFetchMergedBlocksMeta() { - RpcResponseCallback callback = mock(RpcResponseCallback.class); + when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 0)).thenReturn( + new MergedBlockMeta(1, mock(ManagedBuffer.class))); + when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 1)).thenReturn( + new MergedBlockMeta(3, mock(ManagedBuffer.class))); + when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 2)).thenReturn( + new MergedBlockMeta(5, mock(ManagedBuffer.class))); - when(mergedShuffleManager.getChunkCount("app0", 0, 0)).thenReturn(1); - when(mergedShuffleManager.getChunkCount("app0", 0, 1)).thenReturn(3); - when(mergedShuffleManager.getChunkCount("app0", 0, 2)).thenReturn(5); + int[] expectedCount = new int[]{1, 3, 5}; + String appId = "app0"; - ByteBuffer chunksCountsReq = new FetchMergedBlocksMeta("app0", - new String[]{"shuffle_0_-1_0", "shuffle_0_-1_1", "shuffle_0_-1_2"}).toByteBuffer(); - handler.receive(client, chunksCountsReq, callback); for (int reduceId = 0; reduceId < 3; reduceId++) { - verify(mergedShuffleManager, times(1)).getChunkCount("app0", 0, reduceId); + String blockId = "shuffle_0_-1_" + reduceId; + MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); + + handler.getMergedBlockMetaReqHandler() + .receiveMergeBlockMetaReq(client, appId, blockId, callback); + verify(mergedShuffleManager, times(1)).getMergedBlockMeta("app0", 0, reduceId); + + ArgumentCaptor numChunksResponse = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor chunkBitmapResponse = + ArgumentCaptor.forClass(ManagedBuffer.class); + verify(callback, times(1)).onSuccess(numChunksResponse.capture(), + chunkBitmapResponse.capture()); + assertEquals("num chunks in merged block " + reduceId, expectedCount[reduceId], + numChunksResponse.getValue().intValue()); + assertNotNull("chunks bitmap buffer " + reduceId, chunkBitmapResponse.getValue()); } - ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); - verify(callback, times(1)).onSuccess(response.capture()); - verify(callback, never()).onFailure(any()); - - MergedBlocksMeta chunkCounts = - (MergedBlocksMeta) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); - - assertNotNull(chunkCounts.numChunks); - assertEquals(3, chunkCounts.numChunks.length); - assertArrayEquals(new int[]{1, 3, 5}, chunkCounts.numChunks); } @Test @@ -324,7 +328,8 @@ private void verifyBlockChunkFetches(boolean useOpenBlocks) { }; for (int reduceId = 0; reduceId < 2; reduceId++) { for (int chunkId = 0; chunkId < 2; chunkId++) { - when(mergedShuffleManager.getMergedBlockData("app0", 0, reduceId, chunkId)).thenReturn(buffers[reduceId][chunkId]); + when(mergedShuffleManager.getMergedBlockData( + "app0", 0, reduceId, chunkId)).thenReturn(buffers[reduceId][chunkId]); } } @@ -351,7 +356,7 @@ private void verifyBlockChunkFetches(boolean useOpenBlocks) { } assertFalse(bufferIter.hasNext()); - verify(mergedShuffleManager, never()).getChunkCount(anyString(), anyInt(), anyInt()); + verify(mergedShuffleManager, never()).getMergedBlockMeta(anyString(), anyInt(), anyInt()); verify(blockResolver, never()).getBlockData( anyString(), anyString(), anyInt(), anyInt(), anyInt()); verify(mergedShuffleManager, times(1)).getMergedBlockData("app0", 0, 0, 0); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index eb021c32b78de..8e4277ec39b54 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -223,6 +223,26 @@ public void testShuffleBlockChunksFetch() { } } + @Test + public void testShuffleBlockChunkFetchFailure() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffleChunk_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shuffleChunk_0_0_1", null); + blocks.put("shuffleChunk_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks(blocks, blockIds, + new FetchShuffleBlockChunks("app-id", "exec-id", 0, new int[]{0}, + new int[][]{{0, 1, 2}}), conf); + + verify(listener, times(1)).onBlockFetchSuccess( + "shuffleChunk_0_0_0", blocks.get("shuffleChunk_0_0_0")); + verify(listener, times(1)).onBlockFetchFailure( + eq("shuffleChunk_0_0_1"), any()); + verify(listener, times(1)).onBlockFetchSuccess( + "shuffleChunk_0_0_2", blocks.get("shuffleChunk_0_0_2")); + } + /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index fc8f7f8df7f7a..23bf7f4d970e4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -32,6 +32,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,7 +85,7 @@ public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; registerApplication(appId, localDirs); - pushResolver.getChunkCount(appId, 0, 0); + pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); @@ -105,9 +106,8 @@ public void testChunkCountsAndBlockData() throws IOException { ByteBuffer.wrap(new byte[5]) }; pushBlockHelper(appId, pushBlocks, blocks); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(2, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{4, 5}); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}}); removeApplication(appId); } @@ -128,9 +128,8 @@ public void testMultipleBlocksInAChunk() throws IOException { ByteBuffer.wrap(new byte[3]) }; pushBlockHelper(appId, pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(3, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + MergedBlockMeta meta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); removeApplication(appId); } @@ -152,9 +151,8 @@ public void testAppUsingFewerLocalDirs() throws IOException { ByteBuffer.wrap(new byte[3]) }; pushBlockHelper(appId, pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(3, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); removeApplication(appId); } @@ -175,11 +173,18 @@ private void removeApplication(String appId) { } private void validateChunks( - String appId, int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { - for (int i = 0; i < numChunks; i++) { + String appId, int shuffleId, int reduceId, MergedBlockMeta meta, + int[] expectedSizes, int[][] expectedMapsPerChunk) throws IOException { + assertEquals("num chunks", expectedSizes.length, meta.getNumChunks()); + RoaringBitmap[] bitmaps = meta.readChunkBitmaps(); + assertEquals("num of bitmaps", meta.getNumChunks(), bitmaps.length); + for (int i = 0; i < meta.getNumChunks(); i++) { + RoaringBitmap chunkBitmap = bitmaps[i]; + Arrays.stream(expectedMapsPerChunk[i]).forEach(x -> assertTrue(chunkBitmap.contains(x))); + } + for (int i = 0; i < meta.getNumChunks(); i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, - i); + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } From 4381ff3dd090eb6a62b1a73b117bf4d8b896bd62 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 26 Aug 2020 16:28:24 -0700 Subject: [PATCH 11/29] LIHADOOP-55372 reduced the default for minChunkSizeInMergedShuffleFile RB=2253833 G=spark-reviewers R=mshen,vsowrira,mmuralid,yezhou A=mshen --- .../main/java/org/apache/spark/network/util/TransportConf.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 8d8f564eacb99..cc79357fbbd6a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -373,6 +373,6 @@ public boolean useOldFetchProtocol() { */ public int minChunkSizeInMergedShuffleFile() { return Ints.checkedCast(JavaUtils.byteStringAsBytes( - conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4m"))); + conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m"))); } } From dd9958b8df476dd787dede2c65639944bd0a8dd2 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Tue, 8 Sep 2020 23:57:02 -0700 Subject: [PATCH 12/29] LIHADOOP-55315 Avoid network when fetching merged shuffle file in local host with a consistent view of app local dirs among different executors RB=2261073 BUG=LIHADOOP-55315 G=spark-reviewers R=chsingh,mshen,vsowrira,mmuralid A=mmuralid,chsingh --- .../network/shuffle/ExternalBlockHandler.java | 17 +++- .../shuffle/ExternalShuffleBlockResolver.java | 1 + .../shuffle/MergedShuffleFileManager.java | 29 ++++-- .../shuffle/RemoteBlockPushResolver.java | 93 ++++++++++++------- .../shuffle/RemoteBlockPushResolverSuite.java | 36 +++++-- .../network/yarn/YarnShuffleService.java | 13 +-- 6 files changed, 128 insertions(+), 61 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 55a1f42fe6ce2..64e7ae884a973 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -175,6 +176,7 @@ protected void handleMessage( RegisterExecutor msg = (RegisterExecutor) msgObj; checkAuth(client, msg.appId); blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); + mergeManager.registerExecutor(msg.appId, msg.executorInfo.localDirs); callback.onSuccess(ByteBuffer.wrap(new byte[0])); } finally { responseDelayContext.stop(); @@ -190,6 +192,9 @@ protected void handleMessage( GetLocalDirsForExecutors msg = (GetLocalDirsForExecutors) msgObj; checkAuth(client, msg.appId); Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); + if (Arrays.stream(msg.execIds).anyMatch(execId -> execId.isEmpty())) { + localDirs.put("", mergeManager.getMergedBlockDirs(msg.appId)); + } callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); } else if (msgObj instanceof FinalizeShuffleMerge) { @@ -539,10 +544,15 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } @Override - public void registerApplication(String appId, String relativeAppPath) { + public void registerApplication(String appId, String user) { throw new UnsupportedOperationException("Cannot handle shuffle block merge"); } + @Override + public void registerExecutor(String appId, String[] localDirs) { + // No-Op. Do nothing. + } + @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { throw new UnsupportedOperationException("Cannot handle shuffle block merge"); @@ -558,6 +568,11 @@ public ManagedBuffer getMergedBlockData( public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { return null; } + + @Override + public String[] getMergedBlockDirs(String appId) { + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + } } @Override 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 a6bcbb8850566..d0963d28b5d84 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 @@ -371,6 +371,7 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { public Map getLocalDirs(String appId, String[] execIds) { return Arrays.stream(execIds) + .filter(exec -> !exec.isEmpty()) .map(exec -> { ExecutorShuffleInfo info = executors.get(new AppExecId(appId, exec)); if (info == null) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java index 73dc2e207178c..b2812fb7ea99a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -56,17 +56,25 @@ public interface MergedShuffleFileManager { MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException; /** - * Registers an application when it starts. This provides the application specific path + * Registers an application when it starts. It also stores the username which is necessary + * for generating the host local directories for merged shuffle files. + * Right now, this is invoked by YarnShuffleService. + * + * @param appId application ID + * @param user username + */ + void registerApplication(String appId, String user); + + /** + * Registers an executor with its local dir list when it starts. This provides the specific path * so MergedShuffleFileManager knows where to store and look for shuffle data for a - * given application. Right now, this is invoked by YarnShuffleService. + * given application. It is invoked by the RPC call when executor tries to register with the + * local shuffle service. * * @param appId application ID - * @param relativeAppPath The relative path which is application unique. The actual directory - * path is split into 2 parts, where the first half is one of the - * several configured local dirs that're shared across all applications - * and the second half is application unique. + * @param localDirs The list of local dirs that this executor gets granted from NodeManager */ - void registerApplication(String appId, String relativeAppPath); + void registerExecutor(String appId, String[] localDirs); /** * Invoked when an application finishes. This cleans up any remaining metadata associated with @@ -100,4 +108,11 @@ public interface MergedShuffleFileManager { * @return meta information of a merged block */ MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId); + + /** + * Get the local directories which stores the merged shuffle files. + * + * @param appId application ID + */ + String[] getMergedBlockDirs(String appId); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index df13546a17456..e00928a2d2f52 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,7 +37,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; @@ -71,24 +70,21 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); + private static final String MERGE_MANAGER_DIR = "merge_manager"; - private final Path[] localDirs; private final ConcurrentMap appsPathInfo; private final ConcurrentMap partitions; private final Executor directoryCleaner; private final TransportConf conf; private final int minChunkSize; + private final String relativeMergeDirPathPattern; private final LoadingCache indexCache; @SuppressWarnings("UnstableApiUsage") - public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { + public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) { this.conf = conf; - this.localDirs = new Path[localDirs.length]; - for (int i = 0; i < localDirs.length; i++) { - this.localDirs[i] = Paths.get(localDirs[i]); - } this.partitions = Maps.newConcurrentMap(); this.appsPathInfo = Maps.newConcurrentMap(); this.directoryCleaner = Executors.newSingleThreadExecutor( @@ -106,6 +102,7 @@ public ShuffleIndexInformation load(File file) throws IOException { .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) .build(indexCacheLoader); + this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; } /** @@ -205,9 +202,22 @@ private File getFile(String appId, String filename) { AppPathsInfo appPathsInfo = Preconditions.checkNotNull( appsPathInfo.get(appId), "application " + appId + " is not registered or NM was restarted."); - Path[] activeLocalDirs = appPathsInfo.getActiveLocalDirs(localDirs); + Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs); Path localDir = activeLocalDirs[hash % activeLocalDirs.length]; - return new File(localDir.resolve(appPathsInfo.relativeMergeDir).toFile(), filename); + String relativePath = getRelativePath(appPathsInfo.user, appId); + Path filePath = localDir.resolve(relativePath); + File targetFile = new File(filePath.toFile(), filename); + logger.info("Get the file for " + targetFile.getAbsolutePath()); + return targetFile; + } + + private Path[] getActiveLocalDirs(String[] activeLocalDirs) { + return Arrays.stream(activeLocalDirs) + .map(localDir -> Paths.get(localDir)).toArray(Path[]::new); + } + + private String getRelativePath(String user, String appId) { + return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId); } private File getMergedShuffleFile(AppShufflePartitionId id) { @@ -225,6 +235,16 @@ private File getMergedMetaFile(AppShufflePartitionId id) { return getFile(id.appId, metaName); } + @Override + public String[] getMergedBlockDirs(String appId) { + AppPathsInfo appPathsInfo = Preconditions.checkNotNull( + appsPathInfo.get(appId), + "application " + appId + " is not registered or NM was restarted."); + return Arrays.stream(appPathsInfo.activeLocalDirs) + .map(dir -> dir + getRelativePath(appPathsInfo.user, appId)) + .toArray(String[]::new); + } + @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); @@ -249,8 +269,9 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) - .map(dir -> dir.resolve(appPathsInfo.relativeMergeDir)).toArray(Path[]::new); + Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs)) + .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) + .toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } @@ -596,14 +617,23 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } @Override - public void registerApplication(String appId, String relativeAppPath) { - logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); - appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); + public void registerApplication(String appId, String user) { + logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user); + appsPathInfo.put(appId, new AppPathsInfo(user)); } - @VisibleForTesting - public Path[] getLocalDirs() { - return localDirs; + @Override + public void registerExecutor(String appId, String[] localDirs) { + if (logger.isDebugEnabled()) { + logger.debug("register executor with RemoteBlockPushResolver {} {}", + appId, Arrays.toString(localDirs)); + } + Preconditions.checkNotNull(appsPathInfo.get(appId), + "application " + appId + " is not registered or NM was restarted."); + appsPathInfo.compute(appId, + (targetAppId, appPathsInfo) -> + appPathsInfo.updateActiveLocalDirs( + targetAppId, relativeMergeDirPathPattern, localDirs)); } /** @@ -796,28 +826,23 @@ private void writeChunkTracker(int mapId) throws IOException { */ private static class AppPathsInfo { - private final Path relativeMergeDir; - private Path[] activeLocalDirs; + private final String user; + private String[] activeLocalDirs; - AppPathsInfo(Path relativeMergeDir) { - this.relativeMergeDir = Preconditions.checkNotNull( - relativeMergeDir, "relative merge directory path cannot be null"); + AppPathsInfo(String user) { + this.user = Preconditions.checkNotNull(user, "user cannot be null"); } - private Path[] getActiveLocalDirs(Path[] localDirs) { - if (activeLocalDirs != null) { - return activeLocalDirs; - } - synchronized (this) { + private AppPathsInfo updateActiveLocalDirs( + String appId, String relativePathPattern, String[] localDirs) { + if (activeLocalDirs == null) { + String relativePath = String.format(relativePathPattern, user, appId); activeLocalDirs = Arrays.stream(localDirs) - .filter(rootDir -> rootDir.resolve(relativeMergeDir).toFile().exists()) - .toArray(Path[]::new); - if (activeLocalDirs.length == 0) { - throw new RuntimeException( - "Did not find any active local directories wrt " + relativeMergeDir); - } + .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) + .toArray(String[]::new); + logger.info("Updated the active local dirs " + Arrays.toString(activeLocalDirs)); } - return activeLocalDirs; + return this; } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 23bf7f4d970e4..b63a750ac2983 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -52,6 +52,9 @@ public class RemoteBlockPushResolverSuite { private TransportConf conf; private RemoteBlockPushResolver pushResolver; private String[] localDirs; + private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/"; + private final String USER = "testuser"; + private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401"; @Before public void before() throws IOException { @@ -61,7 +64,7 @@ public void before() throws IOException { MapConfigProvider provider = new MapConfigProvider( ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); conf = new TransportConf("shuffle", provider); - pushResolver = new RemoteBlockPushResolver(conf, localDirs); + pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH); } @After @@ -84,7 +87,7 @@ private void cleanupLocalDirs() throws IOException { public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -96,7 +99,8 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -114,7 +118,8 @@ public void testChunkCountsAndBlockData() throws IOException { @Test public void testMultipleBlocksInAChunk() throws IOException { String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -137,7 +142,8 @@ public void testMultipleBlocksInAChunk() throws IOException { public void testAppUsingFewerLocalDirs() throws IOException { String appId = "app_AppUsingFewerLocalDirs"; String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerApplication(appId, activeLocalDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -161,11 +167,23 @@ public void testAppUsingFewerLocalDirs() throws IOException { * This is because when the application gets removed, the directory cleaner removes the merged * data and file in a different thread which can delete the relevant data of a different test. */ - private void registerApplication(String appId, String[] activeLocalDirs) throws IOException { - for (String localDir : activeLocalDirs) { - Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); + private void registerApplication(String appId, String user) throws IOException { + pushResolver.registerApplication(appId, user); + } + + private void registerExecutor(String appId, String[] localDirs) throws IOException { + pushResolver.registerExecutor(appId, localDirs); + for (String localDir : pushResolver.getMergedBlockDirs(appId)) { + Files.createDirectories(Paths.get(localDir)); } - pushResolver.registerApplication(appId, appId + "/merge_manager"); + } + + private String[] prepareBlockManagerLocalDirs(String appId, String user, String[] localDirs){ + return Arrays.stream(localDirs) + .map(localDir -> + localDir + "/" + + String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, appId)) + .toArray(String[]::new); } private void removeApplication(String appId) { diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 0ed63576a75fc..919458ac33d22 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -97,10 +96,7 @@ public class YarnShuffleService extends AuxiliaryService { private static final boolean DEFAULT_STOP_ON_FAILURE = false; // Used by shuffle merge manager to create merged shuffle files. - private static final String YARN_LOCAL_DIRS = "yarn.nodemanager.local-dirs"; - private static final String MERGE_MANAGER_DIR = "merge_manager"; - protected static final String MERGE_DIR_RELATIVE_PATH = - "usercache/%s/appcache/%s/" + MERGE_MANAGER_DIR; + protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/"; // just for testing when you want to find an open port @VisibleForTesting @@ -182,9 +178,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)).sorted() - .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); - shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); + shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a @@ -289,11 +283,10 @@ public void initializeApplication(ApplicationInitializationContext context) { } secretManager.registerApp(appId, shuffleSecret); } - shuffleMergeManager.registerApplication( - appId, String.format(MERGE_DIR_RELATIVE_PATH, context.getUser(), appId)); } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } + shuffleMergeManager.registerApplication(appId, context.getUser()); } @Override From 021dea4d5cfc7c5f6699081568727b79571a17f8 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Fri, 11 Sep 2020 17:03:30 -0700 Subject: [PATCH 13/29] LIHADOOP-55654 Duplicate application init calls trigger NPE and wrong local dirs update in shuffle service. Also fixing a memory leak. RB=2281730 BUG=LIHADOOP-55654 G=spark-reviewers R=vsowrira,chsingh,mshen A=vsowrira,chsingh --- .../shuffle/RemoteBlockPushResolver.java | 21 ++++++++---- .../shuffle/RemoteBlockPushResolverSuite.java | 32 +++++++++++++++++++ 2 files changed, 46 insertions(+), 7 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index e00928a2d2f52..cc316664c2fe2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -207,13 +207,15 @@ private File getFile(String appId, String filename) { String relativePath = getRelativePath(appPathsInfo.user, appId); Path filePath = localDir.resolve(relativePath); File targetFile = new File(filePath.toFile(), filename); - logger.info("Get the file for " + targetFile.getAbsolutePath()); + logger.debug("Get the file for {}", targetFile.getAbsolutePath()); return targetFile; } private Path[] getActiveLocalDirs(String[] activeLocalDirs) { - return Arrays.stream(activeLocalDirs) - .map(localDir -> Paths.get(localDir)).toArray(Path[]::new); + Preconditions.checkNotNull(activeLocalDirs, + "Active local dirs list has not been updated by any executor registration"); + return + Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new); } private String getRelativePath(String user, String appId) { @@ -240,7 +242,11 @@ public String[] getMergedBlockDirs(String appId) { AppPathsInfo appPathsInfo = Preconditions.checkNotNull( appsPathInfo.get(appId), "application " + appId + " is not registered or NM was restarted."); - return Arrays.stream(appPathsInfo.activeLocalDirs) + String[] activeLocalDirs = Preconditions.checkNotNull( + appsPathInfo.get(appId).activeLocalDirs, + "application " + appId + + " active local dirs list has not been updated by any executor registration"); + return Arrays.stream(activeLocalDirs) .map(dir -> dir + getRelativePath(appPathsInfo.user, appId)) .toArray(String[]::new); } @@ -250,7 +256,7 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); // TODO: Change the message when this service is able to handle NM restart AppPathsInfo appPathsInfo = Preconditions.checkNotNull( - appsPathInfo.get(appId), + appsPathInfo.remove(appId), "application " + appId + " is not registered or NM was restarted."); Iterator> iterator = partitions.entrySet().iterator(); @@ -619,7 +625,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc @Override public void registerApplication(String appId, String user) { logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user); - appsPathInfo.put(appId, new AppPathsInfo(user)); + appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user)); } @Override @@ -840,7 +846,8 @@ private AppPathsInfo updateActiveLocalDirs( activeLocalDirs = Arrays.stream(localDirs) .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) .toArray(String[]::new); - logger.info("Updated the active local dirs " + Arrays.toString(activeLocalDirs)); + logger.info("Updated the active local dirs {} for application {}", + Arrays.toString(activeLocalDirs), appId); } return this; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index b63a750ac2983..4f7a56cd41da7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -88,6 +88,7 @@ public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -162,6 +163,37 @@ public void testAppUsingFewerLocalDirs() throws IOException { removeApplication(appId); } + @Test(expected = NullPointerException.class) + public void testUpdateLocalDirsOnlyOnce() throws IOException { + String appId = "app_App1"; + // First app init and executor register will store the active local dirs list + registerApplication(appId, USER); + String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + // Any later app init or executor register from the same application + // won't change the active local dirs list + registerApplication(appId, USER); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + removeApplication(appId); + try { + pushResolver.getMergedBlockDirs(appId); + } catch (Throwable e) { + assertTrue(e.getMessage() + .startsWith("application app_App1 is not registered or NM was restarted.")); + Throwables.propagate(e); + } + } + /** * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. * This is because when the application gets removed, the directory cleaner removes the merged From 5ce02d3b2233ab14cc7610ae129ef1811f677bec Mon Sep 17 00:00:00 2001 From: Min Shen Date: Wed, 23 Sep 2020 02:55:18 -0700 Subject: [PATCH 14/29] Prune changes that should go into a later PR. --- .../network/client/BaseResponseCallback.java | 29 - .../MergedBlockMetaResponseCallback.java | 35 - .../network/client/RpcResponseCallback.java | 5 +- .../spark/network/client/TransportClient.java | 25 +- .../client/TransportResponseHandler.java | 28 +- .../spark/network/crypto/AuthRpcHandler.java | 5 - .../protocol/MergedBlockMetaRequest.java | 86 -- .../protocol/MergedBlockMetaSuccess.java | 92 -- .../spark/network/protocol/Message.java | 5 +- .../network/protocol/MessageDecoder.java | 6 - .../server/AbstractAuthRpcHandler.java | 5 - .../spark/network/server/RpcHandler.java | 18 - .../server/TransportRequestHandler.java | 30 - .../spark/network/util/TransportConf.java | 12 - .../network/TransportRequestHandlerSuite.java | 60 -- .../TransportResponseHandlerSuite.java | 39 - .../protocol/MergedBlockMetaSuccessSuite.java | 106 --- .../spark/network/sasl/SparkSaslSuite.java | 6 +- .../network/shuffle/BlockPushException.java | 8 +- .../network/shuffle/BlockStoreClient.java | 16 - .../network/shuffle/ExternalBlockHandler.java | 142 +-- .../shuffle/ExternalBlockStoreClient.java | 69 -- .../shuffle/MergeFinalizerListener.java | 35 - .../shuffle/MergedBlocksMetaListener.java | 41 - .../shuffle/MergedShuffleFileManager.java | 6 +- .../shuffle/OneForOneBlockFetcher.java | 93 +- .../shuffle/RemoteBlockPushResolver.java | 855 ------------------ .../protocol/AbstractFetchShuffleBlocks.java | 86 -- .../protocol/BlockTransferMessage.java | 4 +- .../protocol/FetchShuffleBlockChunks.java | 123 --- .../shuffle/protocol/FetchShuffleBlocks.java | 43 +- ...uite.java => BlockPushExceptionSuite.java} | 31 +- .../shuffle/ExternalBlockHandlerSuite.java | 129 +-- .../shuffle/OneForOneBlockFetcherSuite.java | 42 - .../shuffle/RemoteBlockPushResolverSuite.java | 253 ------ .../FetchShuffleBlockChunksSuite.java | 42 - .../network/yarn/YarnShuffleService.java | 13 +- .../ExternalShuffleServiceMetricsSuite.scala | 3 +- .../yarn/YarnShuffleServiceMetricsSuite.scala | 3 +- 39 files changed, 106 insertions(+), 2523 deletions(-) delete mode 100644 common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java delete mode 100644 common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java delete mode 100644 common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java delete mode 100644 common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java delete mode 100644 common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java rename common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/{protocol/FetchShuffleBlocksSuite.java => BlockPushExceptionSuite.java} (54%) delete mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java delete mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java deleted file mode 100644 index bb086b3c9a88b..0000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/client/BaseResponseCallback.java +++ /dev/null @@ -1,29 +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.client; - -/** - * A basic callback. This is extended by {@link RpcResponseCallback} and - * {@link MergedBlockMetaResponseCallback} so that both RpcRequests and MergedBlockMetaRequests - * can be handled in {@link TransportResponseHandler} a similar way. - */ -public interface BaseResponseCallback { - - /** Exception either propagated from server or raised on client side. */ - void onFailure(Throwable e); -} diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java deleted file mode 100644 index a0c8edf05c453..0000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/client/MergedBlockMetaResponseCallback.java +++ /dev/null @@ -1,35 +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.client; - -import org.apache.spark.network.buffer.ManagedBuffer; - -/** - * Callback for the result of a single - * {@link org.apache.spark.network.protocol.MergedBlockMetaRequest}. - */ -public interface MergedBlockMetaResponseCallback extends BaseResponseCallback { - /** - * Called upon receipt of a particular merged block meta. - * - * The given buffer will initially have a refcount of 1, but will be release()'d as soon as this - * call returns. You must therefore either retain() the buffer or copy its contents before - * returning. - */ - void onSuccess(int numChunks, ManagedBuffer buffer); -} diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java index a3b8cb1d90a2e..6afc63f71bb3d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java @@ -23,7 +23,7 @@ * Callback for the result of a single RPC. This will be invoked once with either success or * failure. */ -public interface RpcResponseCallback extends BaseResponseCallback { +public interface RpcResponseCallback { /** * Successful serialized result from server. * @@ -31,4 +31,7 @@ public interface RpcResponseCallback extends BaseResponseCallback { * Please copy the content of `response` if you want to use it after `onSuccess` returns. */ void onSuccess(ByteBuffer response); + + /** Exception either propagated from server or raised on client side. */ + void onFailure(Throwable e); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 589810fdc578c..6dcc703e92669 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -200,27 +200,6 @@ public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { return requestId; } - /** - * Sends a FetchMergedBlockMeta message to the server-side. The response of this message is - * either a {@link MergedBlockMetaSuccess} or {@link RpcFailure}. - * - * @param appId applicationId. - * @param mergedBlockId merged block Id. - * @param callback callback the handle the reply. - */ - public void sendMergedBlockMetaReq(String appId, String mergedBlockId, - MergedBlockMetaResponseCallback callback) { - long requestId = requestId(); - if (logger.isTraceEnabled()) { - logger.trace( - "Sending RPC {} to fetch merged block meta to {}", requestId, getRemoteAddress(channel)); - } - handler.addRpcRequest(requestId, callback); - RpcChannelListener listener = new RpcChannelListener(requestId, callback); - channel.writeAndFlush( - new MergedBlockMetaRequest(requestId, appId, mergedBlockId)).addListener(listener); - } - /** * Send data to the remote end as a stream. This differs from stream() in that this is a request * to *send* data to the remote end, not to receive it from the remote. @@ -370,9 +349,9 @@ void handleFailure(String errorMsg, Throwable cause) throws Exception {} private class RpcChannelListener extends StdChannelListener { final long rpcRequestId; - final BaseResponseCallback callback; + final RpcResponseCallback callback; - RpcChannelListener(long rpcRequestId, BaseResponseCallback callback) { + RpcChannelListener(long rpcRequestId, RpcResponseCallback callback) { super("RPC " + rpcRequestId); this.rpcRequestId = rpcRequestId; this.callback = callback; diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 1ee06a3756e49..2f143f77fa4ae 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -33,7 +33,6 @@ import org.apache.spark.network.protocol.ChunkFetchFailure; import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.MergedBlockMetaSuccess; import org.apache.spark.network.protocol.ResponseMessage; import org.apache.spark.network.protocol.RpcFailure; import org.apache.spark.network.protocol.RpcResponse; @@ -57,7 +56,7 @@ public class TransportResponseHandler extends MessageHandler { private final Map outstandingFetches; - private final Map outstandingRpcs; + private final Map outstandingRpcs; private final Queue> streamCallbacks; private volatile boolean streamActive; @@ -82,7 +81,7 @@ public void removeFetchRequest(StreamChunkId streamChunkId) { outstandingFetches.remove(streamChunkId); } - public void addRpcRequest(long requestId, BaseResponseCallback callback) { + public void addRpcRequest(long requestId, RpcResponseCallback callback) { updateTimeOfLastRequest(); outstandingRpcs.put(requestId, callback); } @@ -113,7 +112,7 @@ private void failOutstandingRequests(Throwable cause) { logger.warn("ChunkReceivedCallback.onFailure throws exception", e); } } - for (Map.Entry entry : outstandingRpcs.entrySet()) { + for (Map.Entry entry : outstandingRpcs.entrySet()) { try { entry.getValue().onFailure(cause); } catch (Exception e) { @@ -185,7 +184,7 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof RpcResponse) { RpcResponse resp = (RpcResponse) message; - RpcResponseCallback listener = (RpcResponseCallback) outstandingRpcs.get(resp.requestId); + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); @@ -199,7 +198,7 @@ public void handle(ResponseMessage message) throws Exception { } } else if (message instanceof RpcFailure) { RpcFailure resp = (RpcFailure) message; - BaseResponseCallback listener = outstandingRpcs.get(resp.requestId); + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.errorString); @@ -207,23 +206,6 @@ public void handle(ResponseMessage message) throws Exception { outstandingRpcs.remove(resp.requestId); listener.onFailure(new RuntimeException(resp.errorString)); } - } else if (message instanceof MergedBlockMetaSuccess) { - MergedBlockMetaSuccess resp = (MergedBlockMetaSuccess) message; - MergedBlockMetaResponseCallback listener = - (MergedBlockMetaResponseCallback) outstandingRpcs.get(resp.requestId); - if (listener == null) { - logger.warn( - "Ignoring response for MergedBlockMetaRequest {} from {} ({} bytes) since it is not" - + " outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); - resp.body().release(); - } else { - outstandingRpcs.remove(resp.requestId); - try { - listener.onSuccess(resp.getNumChunks(), resp.body()); - } finally { - resp.body().release(); - } - } } else if (message instanceof StreamResponse) { StreamResponse resp = (StreamResponse) message; Pair entry = streamCallbacks.poll(); diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java index 8f0a40c380219..dd31c955350f1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -138,9 +138,4 @@ protected boolean doAuthChallenge( LOG.debug("Authorization successful for client {}.", channel.remoteAddress()); return true; } - - @Override - public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { - return saslHandler.getMergedBlockMetaReqHandler(); - } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java deleted file mode 100644 index bf32e6134b61d..0000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java +++ /dev/null @@ -1,86 +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.protocol; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -/** - * Request to find the meta information for the specified merged block. The meta information - * contains the number of chunks in the merged blocks and the maps ids in each chunk. - */ -public class MergedBlockMetaRequest extends AbstractMessage implements RequestMessage { - public final long requestId; - public final String appId; - public final String blockId; - - public MergedBlockMetaRequest(long requestId, String appId, String blockId) { - super(null, false); - this.requestId = requestId; - this.appId = appId; - this.blockId = blockId; - } - - @Override - public Type type() { - return Type.MergedBlockMetaRequest; - } - - @Override - public int encodedLength() { - return 8 + Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(blockId); - } - - @Override - public void encode(ByteBuf buf) { - buf.writeLong(requestId); - Encoders.Strings.encode(buf, appId); - Encoders.Strings.encode(buf, blockId); - } - - public static MergedBlockMetaRequest decode(ByteBuf buf) { - long requestId = buf.readLong(); - String appId = Encoders.Strings.decode(buf); - String blockId = Encoders.Strings.decode(buf); - return new MergedBlockMetaRequest(requestId, appId, blockId); - } - - @Override - public int hashCode() { - return Objects.hashCode(requestId, appId, blockId); - } - - @Override - public boolean equals(Object other) { - if (other instanceof MergedBlockMetaRequest) { - MergedBlockMetaRequest o = (MergedBlockMetaRequest) other; - return requestId == o.requestId && Objects.equal(appId, o.appId) && Objects.equal(blockId, - o.blockId); - } - return false; - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("appId", appId) - .add("blockId", blockId) - .toString(); - } -} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java deleted file mode 100644 index a89e9d3103a47..0000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java +++ /dev/null @@ -1,92 +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.protocol; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; - -/** - * Response to {@link MergedBlockMetaRequest} request. - * Note that the server-side encoding of this messages does NOT include the buffer itself. - */ -public class MergedBlockMetaSuccess extends AbstractResponseMessage { - public final long requestId; - public final int numChunks; - - public MergedBlockMetaSuccess( - long requestId, int numChunks, ManagedBuffer chunkBitmapsBuffer) { - super(chunkBitmapsBuffer, true); - this.requestId = requestId; - this.numChunks = numChunks; - } - - @Override - public Type type() { - return Type.MergedBlockMetaSuccess; - } - - @Override - public int hashCode() { - return Objects.hashCode(requestId, numChunks); - } - - @Override - public String toString() { - return com.google.common.base.Objects.toStringHelper(this) - .add("requestId", requestId) - .add("numChunks", numChunks) - .toString(); - } - - @Override - public int encodedLength() { - return 8 + 4; - } - - /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ - @Override - public void encode(ByteBuf buf) { - buf.writeLong(requestId); - buf.writeInt(numChunks); - } - - public int getNumChunks() { - return numChunks; - } - - public ManagedBuffer getChunkBitmapsBuffer() { - return body(); - } - - /** Decoding uses the given ByteBuf as our data, and will retain() it. */ - public static MergedBlockMetaSuccess decode(ByteBuf buf) { - long requestId = buf.readLong(); - int numChunks = buf.readInt(); - buf.retain(); - NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate()); - return new MergedBlockMetaSuccess(requestId, numChunks, managedBuf); - } - - @Override - public ResponseMessage createFailureResponse(String error) { - return new RpcFailure(requestId, error); - } -} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java index 4a2643256fcb4..0ccd70c03aba8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -37,8 +37,7 @@ enum Type implements Encodable { ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), RpcRequest(3), RpcResponse(4), RpcFailure(5), StreamRequest(6), StreamResponse(7), StreamFailure(8), - OneWayMessage(9), UploadStream(10), User(-1), - MergedBlockMetaRequest(11), MergedBlockMetaSuccess(12); + OneWayMessage(9), UploadStream(10), User(-1); private final byte id; @@ -67,8 +66,6 @@ public static Type decode(ByteBuf buf) { case 8: return StreamFailure; case 9: return OneWayMessage; case 10: return UploadStream; - case 11: return MergedBlockMetaRequest; - case 12: return MergedBlockMetaSuccess; case -1: throw new IllegalArgumentException("User type messages cannot be decoded."); default: throw new IllegalArgumentException("Unknown message type: " + id); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java index 98f7f612a486b..bf80aed0afe10 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -83,12 +83,6 @@ private Message decode(Message.Type msgType, ByteBuf in) { case UploadStream: return UploadStream.decode(in); - case MergedBlockMetaRequest: - return MergedBlockMetaRequest.decode(in); - - case MergedBlockMetaSuccess: - return MergedBlockMetaSuccess.decode(in); - default: throw new IllegalArgumentException("Unexpected message type: " + msgType); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java index 95fde677624ff..92eb886283448 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/AbstractAuthRpcHandler.java @@ -104,9 +104,4 @@ public void exceptionCaught(Throwable cause, TransportClient client) { public boolean isAuthenticated() { return isAuthenticated; } - - @Override - public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { - return delegate.getMergedBlockMetaReqHandler(); - } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java index 8a46d37ae25e7..38569baf82bce 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -22,11 +22,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.StreamCallbackWithID; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.protocol.MergedBlockMetaRequest; /** * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. @@ -34,10 +32,6 @@ public abstract class RpcHandler { private static final RpcResponseCallback ONE_WAY_CALLBACK = new OneWayRpcCallback(); - private static final MergedBlockMetaReqHandler NOOP_MERGED_BLOCK_META_REQ_HANDLER = - (client, appId, mergedBlockId, callback) -> { - // do nothing - }; /** * Receive a single RPC message. Any exception thrown while in this method will be sent back to @@ -106,10 +100,6 @@ public void receive(TransportClient client, ByteBuffer message) { receive(client, message, ONE_WAY_CALLBACK); } - public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { - return NOOP_MERGED_BLOCK_META_REQ_HANDLER; - } - /** * Invoked when the channel associated with the given client is active. */ @@ -139,12 +129,4 @@ public void onFailure(Throwable e) { } - /** - * Handler for {@link MergedBlockMetaRequest}. - */ - public interface MergedBlockMetaReqHandler { - void receiveMergeBlockMetaReq( - TransportClient client, String appId, String mergedBlockId, - MergedBlockMetaResponseCallback callback); - } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index a020dd600c6a7..e9dd32f1d181d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -114,8 +114,6 @@ public void handle(RequestMessage request) throws Exception { processStreamRequest((StreamRequest) request); } else if (request instanceof UploadStream) { processStreamUpload((UploadStream) request); - } else if (request instanceof MergedBlockMetaRequest) { - processMergedBlockMetaRequest((MergedBlockMetaRequest) request); } else { throw new IllegalArgumentException("Unknown request type: " + request); } @@ -194,7 +192,6 @@ private void processStreamUpload(final UploadStream req) { // i.e., we always release the original buffer by the time the invocation of this // method ends, instead of by the time we respond it to the client. This is necessary, // otherwise we start seeing memory issues very quickly in benchmarks. - // TODO check if the way metadata buffer is handled can be further improved ByteBuffer meta = cloneBuffer(req.meta.nioByteBuffer()); RpcResponseCallback callback = new RpcResponseCallback() { @Override @@ -296,33 +293,6 @@ private void processOneWayMessage(OneWayMessage req) { } } - private void processMergedBlockMetaRequest(final MergedBlockMetaRequest req) { - try { - rpcHandler.getMergedBlockMetaReqHandler() - .receiveMergeBlockMetaReq(reverseClient, req.appId, req.blockId, - new MergedBlockMetaResponseCallback() { - - @Override - public void onSuccess(int numChunks, ManagedBuffer buffer) { - logger.trace("Sending response for {}: app {} merged blockId {} numChunks {}", - req.requestId, req.appId, req.blockId, numChunks); - respond(new MergedBlockMetaSuccess(req.requestId, numChunks, buffer)); - } - - @Override - public void onFailure(Throwable e) { - logger.trace("Failed to send response for {}: app {} merged blockId {}", - req.requestId, req.appId, req.blockId); - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); - } - }); - } catch (Exception e) { - logger.error("Error while invoking receiveMergeBlockMetaReq() for app {} merged block id {} ", - req.appId, req.blockId, e); - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); - } - } - /** * Make a full copy of a nio ByteBuffer. */ diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index cc79357fbbd6a..646e4278811f4 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -363,16 +363,4 @@ public boolean useOldFetchProtocol() { return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false); } - /** - * The minimum size of a chunk when dividing a merged shuffle file in push based shuffle into - * multiple chunks. This is an optimization so that when push based shuffle merges multiple - * shuffle blocks belonging to the same shuffle partition into a merged shuffle file, it - * divides the merged shuffle file into multiple MB-sized chunks to optimize reading it later. - * A corresponding index file for each merged shuffle file will be generated indicating chunk - * boundaries. - */ - public int minChunkSizeInMergedShuffleFile() { - return Ints.checkedCast(JavaUtils.byteStringAsBytes( - conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m"))); - } } 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 74dcd4ee52ed7..0a6447176237a 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 @@ -17,7 +17,6 @@ package org.apache.spark.network; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -25,19 +24,16 @@ import org.junit.Assert; import org.junit.Test; -import static org.junit.Assert.*; import static org.mockito.Mockito.*; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.protocol.*; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportRequestHandler; public class TransportRequestHandlerSuite { @@ -113,60 +109,4 @@ public void handleStreamRequest() throws Exception { streamManager.connectionTerminated(channel); Assert.assertEquals(0, streamManager.numStreamStates()); } - - @Test - public void handleMergedBlockMetaRequest() throws Exception { - - RpcHandler.MergedBlockMetaReqHandler metaHandler = (client, appId, mergedBlockId, callback) -> { - if (!mergedBlockId.isEmpty()) { - callback.onSuccess(2, mock(ManagedBuffer.class)); - } else { - callback.onFailure(new RuntimeException("empty block")); - } - }; - - RpcHandler rpcHandler = new RpcHandler() { - @Override - public void receive( - TransportClient client, ByteBuffer message, RpcResponseCallback callback) { - } - - @Override - public StreamManager getStreamManager() { - return null; - } - - @Override - public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { - return metaHandler; - } - }; - - Channel channel = mock(Channel.class); - List> responseAndPromisePairs = - new ArrayList<>(); - when(channel.writeAndFlush(any())) - .thenAnswer(invocationOnMock0 -> { - Object response = invocationOnMock0.getArguments()[0]; - ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel); - responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture)); - return channelFuture; - }); - - TransportClient reverseClient = mock(TransportClient.class); - TransportRequestHandler requestHandler = - new TransportRequestHandler(channel, reverseClient, rpcHandler, 2L, null); - - MergedBlockMetaRequest validMetaReq = new MergedBlockMetaRequest(19, "app1", "mergedBlock"); - requestHandler.handle(validMetaReq); - assertEquals(1, responseAndPromisePairs.size()); - assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof MergedBlockMetaSuccess); - assertEquals(2, - ((MergedBlockMetaSuccess) (responseAndPromisePairs.get(0).getLeft())).getNumChunks()); - - MergedBlockMetaRequest invalidMetaReq = new MergedBlockMetaRequest(21, "app1", ""); - requestHandler.handle(invalidMetaReq); - assertEquals(2, responseAndPromisePairs.size()); - assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof RpcFailure); - } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java index 4bcb8bd03951c..b4032c4c3f031 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -23,20 +23,17 @@ import io.netty.channel.Channel; import io.netty.channel.local.LocalChannel; import org.junit.Test; -import org.mockito.ArgumentCaptor; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.ChunkReceivedCallback; -import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.StreamCallback; import org.apache.spark.network.client.TransportResponseHandler; import org.apache.spark.network.protocol.ChunkFetchFailure; import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.MergedBlockMetaSuccess; import org.apache.spark.network.protocol.RpcFailure; import org.apache.spark.network.protocol.RpcResponse; import org.apache.spark.network.protocol.StreamChunkId; @@ -170,40 +167,4 @@ public void failOutstandingStreamCallbackOnException() throws Exception { verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); } - - @Test - public void handleSuccessfulMergedBlockMeta() throws Exception { - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); - handler.addRpcRequest(13, callback); - assertEquals(1, handler.numOutstandingRequests()); - - // This response should be ignored. - handler.handle(new MergedBlockMetaSuccess(22, 2, - new NioManagedBuffer(ByteBuffer.allocate(7)))); - assertEquals(1, handler.numOutstandingRequests()); - - ByteBuffer resp = ByteBuffer.allocate(10); - handler.handle(new MergedBlockMetaSuccess(13, 2, new NioManagedBuffer(resp))); - ArgumentCaptor bufferCaptor = ArgumentCaptor.forClass(NioManagedBuffer.class); - verify(callback, times(1)).onSuccess(eq(2), bufferCaptor.capture()); - assertEquals(resp, bufferCaptor.getValue().nioByteBuffer()); - assertEquals(0, handler.numOutstandingRequests()); - } - - @Test - public void handleFailedMergedBlockMeta() throws Exception { - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); - handler.addRpcRequest(51, callback); - assertEquals(1, handler.numOutstandingRequests()); - - // This response should be ignored. - handler.handle(new RpcFailure(6, "failed")); - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new RpcFailure(51, "failed")); - verify(callback, times(1)).onFailure(any()); - assertEquals(0, handler.numOutstandingRequests()); - } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java deleted file mode 100644 index b6dd371044690..0000000000000 --- a/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java +++ /dev/null @@ -1,106 +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.protocol; - -import java.io.File; -import java.io.FileOutputStream; -import java.nio.channels.FileChannel; -import java.nio.file.Files; -import java.util.List; - -import com.google.common.collect.Lists; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelHandlerContext; -import org.junit.Assert; -import org.junit.Test; -import org.roaringbitmap.RoaringBitmap; - -import static org.mockito.Mockito.*; - -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; -import org.apache.spark.network.util.ByteArrayWritableChannel; -import org.apache.spark.network.util.TransportConf; - -/** - * Test for {@link MergedBlockMetaSuccess}. - */ -public class MergedBlockMetaSuccessSuite { - - @Test - public void testMergedBlocksMetaEncodeDecode() throws Exception { - File chunkMetaFile = new File("target/mergedBlockMetaTest"); - Files.deleteIfExists(chunkMetaFile.toPath()); - RoaringBitmap chunk1 = new RoaringBitmap(); - chunk1.add(1); - chunk1.add(3); - RoaringBitmap chunk2 = new RoaringBitmap(); - chunk2.add(2); - chunk2.add(4); - RoaringBitmap[] expectedChunks = new RoaringBitmap[]{chunk1, chunk2}; - FileChannel fileChannel = new FileOutputStream(chunkMetaFile, true).getChannel(); - ByteBuf buf = Unpooled.buffer(); - for (int i = 0; i < expectedChunks.length; i++) { - Encoders.Bitmaps.encode(buf, expectedChunks[i]); - } - fileChannel.write(buf.nioBuffer()); - fileChannel.close(); - buf.release(); - - TransportConf conf = mock(TransportConf.class); - when(conf.lazyFileDescriptor()).thenReturn(false); - long requestId = 1L; - MergedBlockMetaSuccess expectedMeta = new MergedBlockMetaSuccess(requestId, 2, - new FileSegmentManagedBuffer(conf, chunkMetaFile, 0, chunkMetaFile.length())); - - List out = Lists.newArrayList(); - ChannelHandlerContext context = mock(ChannelHandlerContext.class); - when(context.alloc()).thenReturn(ByteBufAllocator.DEFAULT); - - MessageEncoder.INSTANCE.encode(context, expectedMeta, out); - Assert.assertEquals(1, out.size()); - MessageWithHeader msgWithHeader = (MessageWithHeader) out.remove(0); - - ByteArrayWritableChannel writableChannel = - new ByteArrayWritableChannel((int) msgWithHeader.count()); - while (msgWithHeader.transfered() < msgWithHeader.count()) { - msgWithHeader.transferTo(writableChannel, msgWithHeader.transfered()); - } - ByteBuf messageBuf = Unpooled.wrappedBuffer(writableChannel.getData()); - messageBuf.readLong(); // frame length - MessageDecoder.INSTANCE.decode(mock(ChannelHandlerContext.class), messageBuf, out); - Assert.assertEquals(1, out.size()); - MergedBlockMetaSuccess decoded = (MergedBlockMetaSuccess) out.get(0); - Assert.assertEquals("merged block", expectedMeta.requestId, decoded.requestId); - Assert.assertEquals("num chunks", expectedMeta.getNumChunks(), decoded.getNumChunks()); - - ByteBuf responseBuf = Unpooled.wrappedBuffer(decoded.body().nioByteBuffer()); - RoaringBitmap[] responseBitmaps = new RoaringBitmap[expectedMeta.getNumChunks()]; - for (int i = 0; i < expectedMeta.getNumChunks(); i++) { - responseBitmaps[i] = Encoders.Bitmaps.decode(responseBuf); - } - - Assert.assertEquals( - "num of roaring bitmaps", expectedMeta.getNumChunks(), responseBitmaps.length); - for (int i = 0; i < expectedMeta.getNumChunks(); i++) { - Assert.assertEquals("chunk bitmap " + i, expectedChunks[i], responseBitmaps[i]); - } - Files.delete(chunkMetaFile.toPath()); - } -} diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index cc8b8bcc88e9a..ecaeec98da182 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -357,10 +357,8 @@ public void testRpcHandlerDelegate() throws Exception { public void testDelegates() throws Exception { Method[] rpcHandlerMethods = RpcHandler.class.getDeclaredMethods(); for (Method m : rpcHandlerMethods) { - if (!m.getName().contains("lambda")) { - Method delegate = SaslRpcHandler.class.getMethod(m.getName(), m.getParameterTypes()); - assertNotEquals(delegate.getDeclaringClass(), RpcHandler.class); - } + Method delegate = SaslRpcHandler.class.getMethod(m.getName(), m.getParameterTypes()); + assertNotEquals(delegate.getDeclaringClass(), RpcHandler.class); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java index beb3d694bb77d..1c8e145d1eb2b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java @@ -56,7 +56,13 @@ public static BlockPushException decodeException(String message) { BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(rawBuffer); if (msgObj instanceof PushBlockStream) { PushBlockStream header = (PushBlockStream) msgObj; - return new BlockPushException(header, StandardCharsets.UTF_8.decode(rawBuffer).toString()); + // When decoding the header, the rawBuffer's position is not updated since it was + // consumed via netty's ByteBuf. Updating the rawBuffer's position here to retrieve + // the remaining exception message. + ByteBuffer remainingBuffer = (ByteBuffer) rawBuffer.position(rawBuffer.position() + + header.encodedLength() + 1); + return new BlockPushException(header, + StandardCharsets.UTF_8.decode(remainingBuffer).toString()); } else { throw new UnsupportedOperationException(String.format("Cannot decode the header. " + "Expected PushBlockStream but got %s instead", msgObj.getClass().getSimpleName())); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index 13b2779aa79e2..1161a3351b3a9 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -155,20 +155,4 @@ public void pushBlocks( BlockFetchingListener listener) { throw new UnsupportedOperationException(); } - - /** - * Get the meta information of a merged block from the remote shuffle service. - * - * @param host the host of the remote node. - * @param port the port of the remote node. - * @param block block id of the merged shuffle block. - * @param listener the listener to receive chunk counts. - */ - public void getMergedBlockMeta( - String host, - int port, - String block, - MergedBlocksMetaListener listener) { - throw new UnsupportedOperationException(); - } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 64e7ae884a973..b8cb581ab8e07 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -33,7 +33,6 @@ import com.codahale.metrics.Timer; import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; -import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.StreamCallbackWithID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,11 +56,8 @@ * Blocks are registered with the "one-for-one" strategy, meaning each Transport-layer Chunk * is equivalent to one block. */ -public class ExternalBlockHandler extends RpcHandler - implements RpcHandler.MergedBlockMetaReqHandler { +public class ExternalBlockHandler extends RpcHandler { private static final Logger logger = LoggerFactory.getLogger(ExternalBlockHandler.class); - private static final String SHUFFLE_BLOCK_PREFIX = "shuffle"; - private static final String SHUFFLE_CHUNK_PREFIX = "shuffleChunk"; @VisibleForTesting final ExternalShuffleBlockResolver blockManager; @@ -72,7 +68,8 @@ public class ExternalBlockHandler extends RpcHandler public ExternalBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { this(new OneForOneStreamManager(), - new ExternalShuffleBlockResolver(conf, registeredExecutorFile), new NoOpMergedShuffleFileManager()); + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), + new NoOpMergedShuffleFileManager()); } public ExternalBlockHandler( @@ -96,7 +93,7 @@ public ExternalBlockHandler( this(streamManager, blockManager, new NoOpMergedShuffleFileManager()); } - /** Enables mocking out the StreamManager and BlockManager. */ + /** Enables mocking out the StreamManager, BlockManager, and MergeManager. */ @VisibleForTesting public ExternalBlockHandler( OneForOneStreamManager streamManager, @@ -133,22 +130,24 @@ protected void handleMessage( BlockTransferMessage msgObj, TransportClient client, RpcResponseCallback callback) { - if (msgObj instanceof AbstractFetchShuffleBlocks || msgObj instanceof OpenBlocks) { + if (msgObj instanceof FetchShuffleBlocks || msgObj instanceof OpenBlocks) { final Timer.Context responseDelayContext = metrics.openBlockRequestLatencyMillis.time(); try { int numBlockIds; long streamId; - if (msgObj instanceof AbstractFetchShuffleBlocks) { - AbstractFetchShuffleBlocks msg = (AbstractFetchShuffleBlocks) msgObj; + if (msgObj instanceof FetchShuffleBlocks) { + FetchShuffleBlocks msg = (FetchShuffleBlocks) msgObj; checkAuth(client, msg.appId); - numBlockIds = ((AbstractFetchShuffleBlocks) msgObj).getNumBlocks(); - Iterator iterator; - if (msgObj instanceof FetchShuffleBlocks) { - iterator = new ShuffleManagedBufferIterator((FetchShuffleBlocks)msgObj); + numBlockIds = 0; + if (msg.batchFetchEnabled) { + numBlockIds = msg.mapIds.length; } else { - iterator = new ShuffleChunkManagedBufferIterator((FetchShuffleBlockChunks)msgObj); + for (int[] ids: msg.reduceIds) { + numBlockIds += ids.length; + } } - streamId = streamManager.registerStream(client.getClientId(), iterator, client.getChannel()); + streamId = streamManager.registerStream(client.getClientId(), + new ShuffleManagedBufferIterator(msg), client.getChannel()); } else { // For the compatibility with the old version, still keep the support for OpenBlocks. OpenBlocks msg = (OpenBlocks) msgObj; @@ -169,6 +168,7 @@ protected void handleMessage( } finally { responseDelayContext.stop(); } + } else if (msgObj instanceof RegisterExecutor) { final Timer.Context responseDelayContext = metrics.registerExecutorRequestLatencyMillis.time(); @@ -192,7 +192,7 @@ protected void handleMessage( GetLocalDirsForExecutors msg = (GetLocalDirsForExecutors) msgObj; checkAuth(client, msg.appId); Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); - if (Arrays.stream(msg.execIds).anyMatch(execId -> execId.isEmpty())) { + if (Arrays.stream(msg.execIds).anyMatch(String::isEmpty)) { localDirs.put("", mergeManager.getMergedBlockDirs(msg.appId)); } callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); @@ -216,35 +216,6 @@ protected void handleMessage( } } - @Override - public void receiveMergeBlockMetaReq( - TransportClient client, String appId, String mergedBlockId, - MergedBlockMetaResponseCallback callback) { - - final Timer.Context responseDelayContext = metrics.fetchMergedBlocksMetaLatencyMillis.time(); - try { - checkAuth(client, appId); - String[] blockIdParts = mergedBlockId.split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) { - throw new IllegalArgumentException( - "Unexpected shuffle block id format: " + mergedBlockId); - } - MergedBlockMeta mergedMeta = - mergeManager.getMergedBlockMeta(appId, Integer.parseInt(blockIdParts[1]), - Integer.parseInt(blockIdParts[3])); - logger.debug( - "Merged block chunks {} : {} ", mergedBlockId, mergedMeta.getNumChunks()); - callback.onSuccess(mergedMeta.getNumChunks(), mergedMeta.getChunksBitmapBuffer()); - } finally { - responseDelayContext.stop(); - } - } - - @Override - public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() { - return this; - } - @Override public void exceptionCaught(Throwable cause, TransportClient client) { metrics.caughtExceptions.inc(); @@ -309,8 +280,6 @@ public class ShuffleMetrics implements MetricSet { private final Timer openBlockRequestLatencyMillis = new Timer(); // Time latency for executor registration latency in ms private final Timer registerExecutorRequestLatencyMillis = new Timer(); - // Time latency for processing fetch merged blocks meta request latency in ms - private final Timer fetchMergedBlocksMetaLatencyMillis = new Timer(); // Time latency for processing finalize shuffle merge request latency in ms private final Timer finalizeShuffleMergeLatencyMillis = new Timer(); // Block transfer rate in byte per second @@ -324,7 +293,6 @@ public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); - allMetrics.put("fetchMergedBlocksMetaLatencyMillis", fetchMergedBlocksMetaLatencyMillis); allMetrics.put("finalizeShuffleMergeLatencyMillis", finalizeShuffleMergeLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", @@ -344,36 +312,19 @@ private class ManagedBufferIterator implements Iterator { private int index = 0; private final Function blockDataForIndexFn; private final int size; - private final boolean requestForMergedBlockChunks; ManagedBufferIterator(OpenBlocks msg) { String appId = msg.appId; String execId = msg.execId; String[] blockIds = msg.blockIds; String[] blockId0Parts = blockIds[0].split("_"); - if (blockId0Parts.length == 4 - && (blockId0Parts[0].equals(SHUFFLE_BLOCK_PREFIX) - || blockId0Parts[0].equals(SHUFFLE_CHUNK_PREFIX))) { + if (blockId0Parts.length == 4 && blockId0Parts[0].equals("shuffle")) { final int shuffleId = Integer.parseInt(blockId0Parts[1]); - requestForMergedBlockChunks = blockId0Parts[0].equals(SHUFFLE_CHUNK_PREFIX); final int[] mapIdAndReduceIds = shuffleMapIdAndReduceIds(blockIds, shuffleId); size = mapIdAndReduceIds.length; - blockDataForIndexFn = - index -> { - if (requestForMergedBlockChunks) { - return mergeManager.getMergedBlockData( - msg.appId, shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); - } else { - return blockManager.getBlockData( - msg.appId, - msg.execId, - shuffleId, - mapIdAndReduceIds[index], - mapIdAndReduceIds[index + 1]); - } - }; + blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, shuffleId, + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); } else if (blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { - requestForMergedBlockChunks = false; final int[] rddAndSplitIds = rddAndSplitIds(blockIds); size = rddAndSplitIds.length; blockDataForIndexFn = index -> blockManager.getRddBlockData(appId, execId, @@ -400,18 +351,14 @@ 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 || - (!requestForMergedBlockChunks && !blockIdParts[0].equals(SHUFFLE_BLOCK_PREFIX)) || - (requestForMergedBlockChunks && !blockIdParts[0].equals(SHUFFLE_CHUNK_PREFIX))) { + if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[i]); } if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockIds[i]); } - // For regular blocks this is mapId. For chunks this is reduceId. mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); - // For regular blocks this is reduceId. For chunks this is chunkId. mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } return mapIdAndReduceIds; @@ -484,52 +431,9 @@ public ManagedBuffer next() { } } - private class ShuffleChunkManagedBufferIterator implements Iterator { - - private int reduceIdx = 0; - private int chunkIdx = 0; - - private final String appId; - private final String execId; - private final int shuffleId; - private final int[] reduceIds; - private final int[][] chunkIds; - - ShuffleChunkManagedBufferIterator(FetchShuffleBlockChunks msg) { - appId = msg.appId; - execId = msg.execId; - shuffleId = msg.shuffleId; - reduceIds = msg.reduceIds; - chunkIds = msg.chunkIds; - } - - @Override - public boolean hasNext() { - // reduceIds.length must equal to chunkIds.length, and the passed in FetchShuffleBlockChunks - // must have non-empty reduceIds and chunkIds, see the checking logic in - // OneForOneBlockFetcher. - assert(reduceIds.length != 0 && reduceIds.length == chunkIds.length); - return reduceIdx < reduceIds.length && chunkIdx < chunkIds[reduceIdx].length; - } - - @Override - public ManagedBuffer next() { - ManagedBuffer block = mergeManager.getMergedBlockData( - appId, shuffleId, reduceIds[reduceIdx], chunkIds[reduceIdx][chunkIdx]); - if (chunkIdx < chunkIds[reduceIdx].length - 1) { - chunkIdx += 1; - } else { - chunkIdx = 0; - reduceIdx += 1; - } - metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); - return block; - } - } - /** - * Dummy implementation of merged shuffle file manager. Suitable for when this feature - * is not turned on. + * Dummy implementation of merged shuffle file manager. Suitable for when push-based shuffle + * is not enabled. */ private static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index cbde0d48bbad8..306d27fa6fe00 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -31,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; @@ -161,74 +160,6 @@ public void pushBlocks( } } - @Override - public void getMergedBlockMeta( - String host, - int port, - String mergedBlockId, - MergedBlocksMetaListener listener) { - checkInit(); - logger.debug("Get merged blocks meta from {}:{} for block {}", host, port, mergedBlockId); - try { - TransportClient client = clientFactory.createClient(host, port); - client.sendMergedBlockMetaReq(appId, mergedBlockId, new MergedBlockMetaResponseCallback() { - @Override - public void onSuccess(int numChunks, ManagedBuffer buffer) { - logger.trace("Successfully got merged block meta for {}", mergedBlockId); - listener.onSuccess(mergedBlockId, new MergedBlockMeta(numChunks, buffer)); - } - - @Override - public void onFailure(Throwable e) { - logger.error("Failed while getting merged block meta", e); - listener.onFailure(mergedBlockId, e); - } - }); - } catch (Exception e) { - logger.error("Exception while getting merged block meta", e); - listener.onFailure(mergedBlockId, e); - } - } - - /** - * Invoked by Spark driver to notify external shuffle services to finalize the shuffle merge - * for a given shuffle. This allows the driver to start the shuffle reducer stage after properly - * finishing the shuffle merge process associated with the shuffle mapper stage. - * - * @param host host of shuffle server - * @param port port of shuffle server. - * @param shuffleId shuffle ID of the shuffle to be finalized - * @param listener the listener to receive MergeStatuses - */ - // TODO Might be better create a separate shuffle client similar to MesosExternalShuffleClient, - // TODO as this is going to be used by the driver, to avoid having to initialize an - // TODO ExternalShuffleClient. - public void finalizeShuffleMerge( - String host, - int port, - int shuffleId, - MergeFinalizerListener listener) { - checkInit(); - try { - TransportClient client = clientFactory.createClient(host, port); - ByteBuffer finalizeShuffleMerge = new FinalizeShuffleMerge(appId, shuffleId).toByteBuffer(); - client.sendRpc(finalizeShuffleMerge, new RpcResponseCallback() { - @Override - public void onSuccess(ByteBuffer response) { - listener.onShuffleMergeSuccess( - (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response)); - } - - @Override - public void onFailure(Throwable e) { - listener.onShuffleMergeFailure(e); - } - }); - } catch (Exception e) { - logger.error("Exception while sending finalizeShuffleMerge request", e); - } - } - @Override public MetricSet shuffleMetrics() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java deleted file mode 100644 index 2d353bf2c5350..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergeFinalizerListener.java +++ /dev/null @@ -1,35 +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; - -import java.util.EventListener; - -import org.apache.spark.network.shuffle.protocol.MergeStatuses; - -public interface MergeFinalizerListener extends EventListener { - /** - * Called once upon successful response on finalize shuffle merge on a remote shuffle service. - * The returned {@link MergeStatuses} is passed to the listener for further processing - */ - void onShuffleMergeSuccess(MergeStatuses statuses); - - /** - * Called once upon failure - */ - void onShuffleMergeFailure(Throwable e); -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java deleted file mode 100644 index 55ea9bf81a24e..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlocksMetaListener.java +++ /dev/null @@ -1,41 +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; - -import java.util.EventListener; - -/** - * Listener for receiving success or failure events when fetching meta of merged blocks. - */ -public interface MergedBlocksMetaListener extends EventListener { - - /** - * Called after successfully receiving the meta of merged blocks. Currently, the meta only - * includes the count of chunks in a merged block. - * @param mergedBlockId merged block Id. - * @param meta contains meta information of a merged block. - */ - void onSuccess(String mergedBlockId, MergedBlockMeta meta); - - /** - * Called when there is an exception while fetching the meta of merged blocks. - * @param mergedBlockId merged block Id. - * @param exception exception getting chunk counts. - */ - void onFailure(String mergedBlockId, Throwable exception); -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java index b2812fb7ea99a..da96ef4cae67d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -28,7 +28,7 @@ /** * The MergedShuffleFileManager is used to process push based shuffle when enabled. It works - * along side {@link ExternalShuffleBlockHandler} and serves as an RPCHandler for + * along side {@link ExternalBlockHandler} and serves as an RPCHandler for * {@link org.apache.spark.network.server.RpcHandler#receiveStream}, where it processes the * remotely pushed streams of shuffle blocks to merge them into merged shuffle files. Right * now, push based shuffle can only be enabled when external shuffle service in YARN mode @@ -82,9 +82,7 @@ public interface MergedShuffleFileManager { * * @param appId application ID * @param cleanupLocalDirs flag indicating whether MergedShuffleFileManager should handle - * deletion of local dirs itself. Ideally, we should be able to delegate - * to YARN to handle local dir deletion in YARN mode. This does not work - * as expected yet. See LIHADOOP-52202. + * deletion of local dirs itself. */ void applicationRemoved(String appId, boolean cleanupLocalDirs); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index c22622b14ae31..ec2e3dce661d9 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -25,7 +25,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,7 +34,6 @@ import org.apache.spark.network.client.StreamCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.OneForOneStreamManager; -import org.apache.spark.network.shuffle.protocol.AbstractFetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; @@ -54,9 +52,6 @@ public class OneForOneBlockFetcher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); - private static final String SHUFFLE_BLOCK_PREFIX = "shuffle"; - private static final String SHUFFLE_CHUNK_PREFIX = "shuffleChunk"; - private final TransportClient client; private final BlockTransferMessage message; private final String[] blockIds; @@ -94,68 +89,22 @@ public OneForOneBlockFetcher( if (blockIds.length == 0) { throw new IllegalArgumentException("Zero-sized blockIds array"); } - if (!transportConf.useOldFetchProtocol() && areShuffleBlocksOrChunks(blockIds)) { - this.message = createFetchShuffleBlocksOrChunksMsg(appId, execId, blockIds); + if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { + this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); } else { this.message = new OpenBlocks(appId, execId, blockIds); } } - /** - * Check if the array of block IDs are all shuffle block IDs. With push based shuffle, - * the shuffle block ID could be either unmerged shuffle block IDs or merged shuffle chunk - * IDs. For a given stream of shuffle blocks to be fetched in one request, they would be either - * all unmerged shuffle blocks or all merged shuffle chunks. - * @param blockIds block ID array - * @return whether the array contains only shuffle block IDs - */ - private boolean areShuffleBlocksOrChunks(String[] blockIds) { + private boolean isShuffleBlocks(String[] blockIds) { for (String blockId : blockIds) { - if (!blockId.startsWith(SHUFFLE_BLOCK_PREFIX) && - !blockId.startsWith(SHUFFLE_CHUNK_PREFIX)) { + if (!blockId.startsWith("shuffle_")) { return false; } } return true; } - /** Creates either a {@link FetchShuffleBlocks} or {@link FetchShuffleBlockChunks} message. */ - private AbstractFetchShuffleBlocks createFetchShuffleBlocksOrChunksMsg( - String appId, String execId, String[] blockIds) { - if (blockIds[0].startsWith(SHUFFLE_CHUNK_PREFIX)) { - return createFetchShuffleBlockChunksMsg(appId, execId, blockIds); - } else { - return createFetchShuffleBlocksMsg(appId, execId, blockIds); - } - } - - /** - * Creates FetchShuffleBlockChunks message. - */ - private FetchShuffleBlockChunks createFetchShuffleBlockChunksMsg( - String appId, String execId, String[] blockIds) { - String[] firstBlock = splitBlockId(blockIds[0]); - int shuffleId = Integer.parseInt(firstBlock[1]); - HashMap> reduceIdsToChunkIds = new HashMap<>(); - for (String blockId : blockIds) { - String[] blockIdParts = splitBlockId(blockId); - if (Integer.parseInt(blockIdParts[1]) != shuffleId) { - throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); - } - int reduceId = Integer.parseInt(blockIdParts[2]); - if (!reduceIdsToChunkIds.containsKey(reduceId)) { - reduceIdsToChunkIds.put(reduceId, new ArrayList<>()); - } - reduceIdsToChunkIds.get(reduceId).add(Integer.parseInt(blockIdParts[3])); - } - int[] reduceIds = Ints.toArray(reduceIdsToChunkIds.keySet()); - int[][] chunkIdArr = new int[reduceIds.length][]; - for (int i = 0; i < chunkIdArr.length; i++) { - chunkIdArr[i] = Ints.toArray(reduceIdsToChunkIds.get(reduceIds[i])); - } - return new FetchShuffleBlockChunks(appId, execId, shuffleId, reduceIds, chunkIdArr); - } - /** * Analyze the pass in blockIds and create FetchShuffleBlocks message. * The blockIds has been sorted by mapId and reduceId. It's produced in @@ -201,8 +150,7 @@ private String[] splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); // For batch block id, the format contains shuffleId, mapId, begin reduceId, end reduceId. // For single block id, the format contains shuffleId, mapId, educeId. - if (blockIdParts.length < 4 || blockIdParts.length > 5 - || !(blockIdParts[0].equals("shuffle") || blockIdParts[0].equals(SHUFFLE_CHUNK_PREFIX))) { + if (blockIdParts.length < 4 || blockIdParts.length > 5 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + blockId); } @@ -219,16 +167,9 @@ public void onSuccess(int chunkIndex, ManagedBuffer buffer) { @Override public void onFailure(int chunkIndex, Throwable e) { - // If failed block is a merged block, we only fail this block and do not - // fail the remaining blocks. The failed merged block will be retried by - // falling back to fetching the original unmerged blocks. - if ((blockIds[chunkIndex]).startsWith(SHUFFLE_CHUNK_PREFIX)) { - failSingleBlockChunk(blockIds[chunkIndex], e); - } else { - // On receipt of a failure, fail every block from chunkIndex onwards. - String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); - failRemainingBlocks(remainingBlockIds, e); - } + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, e); } } @@ -280,14 +221,6 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { } } - private void failSingleBlockChunk(String shuffleBlockChunkId, Throwable e) { - try { - listener.onBlockFetchFailure(shuffleBlockChunkId, e); - } catch (Exception e2) { - logger.error("Error in block fetch failure callback", e2); - } - } - private class DownloadCallback implements StreamCallback { private DownloadFileWritableChannel channel = null; @@ -318,13 +251,9 @@ public void onComplete(String streamId) throws IOException { @Override public void onFailure(String streamId, Throwable cause) throws IOException { channel.close(); - if ((blockIds[chunkIndex]).startsWith(SHUFFLE_CHUNK_PREFIX)) { - failSingleBlockChunk(blockIds[chunkIndex], cause); - } else { - // On receipt of a failure, fail every block from chunkIndex onwards. - String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); - failRemainingBlocks(remainingBlockIds, cause); - } + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, cause); targetFile.delete(); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java deleted file mode 100644 index cc316664c2fe2..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ /dev/null @@ -1,855 +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; - -import java.io.BufferedOutputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; - -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.cache.Weigher; -import com.google.common.collect.Maps; -import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import org.roaringbitmap.RoaringBitmap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.StreamCallbackWithID; -import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; -import org.apache.spark.network.shuffle.protocol.MergeStatuses; -import org.apache.spark.network.shuffle.protocol.PushBlockStream; -import org.apache.spark.network.util.JavaUtils; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportConf; - -/** - * An implementation of MergedShuffleFileManager that provides the most essential shuffle - * service processing logic to support push based shuffle. - */ -public class RemoteBlockPushResolver implements MergedShuffleFileManager { - - private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); - private static final String MERGE_MANAGER_DIR = "merge_manager"; - - private final ConcurrentMap appsPathInfo; - private final ConcurrentMap partitions; - - private final Executor directoryCleaner; - private final TransportConf conf; - private final int minChunkSize; - private final String relativeMergeDirPathPattern; - - private final LoadingCache indexCache; - - @SuppressWarnings("UnstableApiUsage") - public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) { - this.conf = conf; - this.partitions = Maps.newConcurrentMap(); - this.appsPathInfo = Maps.newConcurrentMap(); - this.directoryCleaner = Executors.newSingleThreadExecutor( - // Add `spark` prefix because it will run in NM in Yarn mode. - NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); - this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); - String indexCacheSize = conf.get("spark.shuffle.service.mergedIndex.cache.size", "100m"); - CacheLoader indexCacheLoader = - new CacheLoader() { - public ShuffleIndexInformation load(File file) throws IOException { - return new ShuffleIndexInformation(file); - } - }; - indexCache = CacheBuilder.newBuilder() - .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) - .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) - .build(indexCacheLoader); - this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; - } - - /** - * Given an ID that uniquely identifies a given shuffle partition of an application, retrieves - * the associated metadata. If not present and the corresponding merged shuffle does not exist, - * initializes the metadata. - */ - private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( - AppShufflePartitionId id) { - return partitions.computeIfAbsent(id, key -> { - // It only gets here when the key is not present in the map. This could either - // be the first time the merge manager receives a pushed block for a given application - // shuffle partition, or after the merged shuffle file is finalized. We handle these - // two cases accordingly by checking if the file already exists. - File mergedShuffleFile = getMergedShuffleFile(key); - File mergedIndexFile = getMergedIndexFile(id); - File mergedMetaFile = getMergedMetaFile(id); - try { - if (mergedShuffleFile.exists()) { - return null; - } else { - return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile, mergedMetaFile); - } - } catch (IOException e) { - logger.error( - "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and " - + "meta file {}", key, mergedIndexFile.getAbsolutePath(), - mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath()); - throw new RuntimeException(String.format( - "Cannot initialize merged shuffle partition %s", key.toString()), e); - } - }); - } - - @Override - public MergedBlockMeta getMergedBlockMeta( - String appId, - int shuffleId, - int reduceId) { - AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); - File indexFile = getMergedIndexFile(id); - if (!indexFile.exists()) { - throw new RuntimeException( - String.format("Application merged shuffle index file is not found (id=%s)", - id.toString())); - } - int size = (int) indexFile.length(); - // First entry is the zero offset - int numChunks = (size / Long.BYTES) - 1; - File metaFile = getMergedMetaFile(id); - if (!metaFile.exists()) { - throw new RuntimeException( - String.format("Application merged shuffle meta file is not found (id=%s)", - id.toString())); - } - FileSegmentManagedBuffer chunkBitMaps = - new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length()); - logger.trace( - "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks); - return new MergedBlockMeta(numChunks, chunkBitMaps); - } - - @SuppressWarnings("UnstableApiUsage") - @Override - public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) { - AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); - File mergedShuffleFile = getMergedShuffleFile(id); - if (!mergedShuffleFile.exists()) { - throw new RuntimeException( - String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(), - id.toString())); - } - File indexFile = getMergedIndexFile(id); - try { - // If we get here, the merged shuffle file should have been properly finalized. Thus we can - // use the file length to determine the size of the merged shuffle block. - ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile); - ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId); - return new FileSegmentManagedBuffer( - conf, - mergedShuffleFile, - shuffleIndexRecord.getOffset(), - shuffleIndexRecord.getLength()); - } catch (ExecutionException e) { - throw new RuntimeException("Failed to open file: " + indexFile, e); - } - } - - /** - * The logic here is consistent with - * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile - */ - // TODO should we use subDirsPerLocalDir to potentially reduce inode size? - private File getFile(String appId, String filename) { - int hash = JavaUtils.nonNegativeHash(filename); - // TODO: Change the message when this service is able to handle NM restart - AppPathsInfo appPathsInfo = Preconditions.checkNotNull( - appsPathInfo.get(appId), - "application " + appId + " is not registered or NM was restarted."); - Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs); - Path localDir = activeLocalDirs[hash % activeLocalDirs.length]; - String relativePath = getRelativePath(appPathsInfo.user, appId); - Path filePath = localDir.resolve(relativePath); - File targetFile = new File(filePath.toFile(), filename); - logger.debug("Get the file for {}", targetFile.getAbsolutePath()); - return targetFile; - } - - private Path[] getActiveLocalDirs(String[] activeLocalDirs) { - Preconditions.checkNotNull(activeLocalDirs, - "Active local dirs list has not been updated by any executor registration"); - return - Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new); - } - - private String getRelativePath(String user, String appId) { - return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId); - } - - private File getMergedShuffleFile(AppShufflePartitionId id) { - String fileName = id.generateFileName(); - return getFile(id.appId, fileName); - } - - private File getMergedIndexFile(AppShufflePartitionId id) { - String indexName = id.generateIndexFileName(); - return getFile(id.appId, indexName); - } - - private File getMergedMetaFile(AppShufflePartitionId id) { - String metaName = id.generateMetaFileName(); - return getFile(id.appId, metaName); - } - - @Override - public String[] getMergedBlockDirs(String appId) { - AppPathsInfo appPathsInfo = Preconditions.checkNotNull( - appsPathInfo.get(appId), - "application " + appId + " is not registered or NM was restarted."); - String[] activeLocalDirs = Preconditions.checkNotNull( - appsPathInfo.get(appId).activeLocalDirs, - "application " + appId + - " active local dirs list has not been updated by any executor registration"); - return Arrays.stream(activeLocalDirs) - .map(dir -> dir + getRelativePath(appPathsInfo.user, appId)) - .toArray(String[]::new); - } - - @Override - public void applicationRemoved(String appId, boolean cleanupLocalDirs) { - logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - // TODO: Change the message when this service is able to handle NM restart - AppPathsInfo appPathsInfo = Preconditions.checkNotNull( - appsPathInfo.remove(appId), - "application " + appId + " is not registered or NM was restarted."); - Iterator> iterator = - partitions.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - AppShufflePartitionId partitionId = entry.getKey(); - AppShufflePartitionInfo partition = entry.getValue(); - if (appId.equals(partitionId.appId)) { - iterator.remove(); - try { - partition.channel.close(); - } catch (IOException e) { - logger.error("Error closing merged shuffle file for {}", partitionId); - } - } - } - - if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs)) - .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) - .toArray(Path[]::new); - directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); - } - } - - /** - * Synchronously delete local dirs, executed in a separate thread. - */ - private void deleteExecutorDirs(Path[] dirs) { - for (Path localDir : dirs) { - try { - if (Files.exists(localDir)) { - JavaUtils.deleteRecursively(localDir.toFile()); - logger.debug("Successfully cleaned up directory: {}", localDir); - } - } catch (Exception e) { - logger.error("Failed to delete directory: {}", localDir, e); - } - } - } - - @Override - public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { - // Retrieve merged shuffle file metadata - String[] blockIdParts = msg.blockId.split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { - throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId); - } - AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId, - Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3])); - int mapId = Integer.parseInt(blockIdParts[2]); - AppShufflePartitionInfo partitionInfoBeforeCheck = - getOrCreateAppShufflePartitionInfo(partitionId); - - // Here partitionInfo will be null in 2 cases: - // 1) The request is received for a block that has already been merged, this is possible due - // to the retry logic. - // 2) The request is received after the merged shuffle is finalized, thus is too late. - // - // For case 1, we will drain the data in the channel and just respond success - // to the client. This is required because the response of the previously merged - // block will be ignored by the client, per the logic in RetryingBlockFetcher. - // Note that the netty server should receive data for a given block id only from 1 channel - // at any time. The block should be pushed only from successful maps, thus there should be - // only 1 source for a given block at any time. Although the netty client might retry sending - // this block to the server multiple times, the data of the same block always arrives from the - // same channel thus the server should have already processed the previous request of this - // block before seeing it again in the channel. This guarantees that we can simply just - // check the bitmap to determine if a block is a duplicate or not. - // - // For case 2, we will also drain the data in the channel, but throw an exception in - // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way, - // the client will be notified of the failure but the channel will remain active. Keeping - // the channel alive is important because the same channel could be reused by multiple map - // tasks in the executor JVM, which belongs to different stages. While one of the shuffles - // in these stages is finalized, the others might still be active. Tearing down the channel - // on the server side will disrupt these other on-going shuffle merges. It's also important - // to notify the client of the failure, so that it can properly halt pushing the remaining - // blocks upon receiving such failures to preserve resources on the server/client side. - // - // Speculative execution would also raise a possible scenario with duplicate blocks. Although - // speculative execution would kill the slower task attempt, leading to only 1 task attempt - // succeeding in the end, there is no guarantee that only one copy of the block will be - // pushed. This is due to our handling of block push process outside of the map task, thus - // it is possible for the speculative task attempt to initiate the block push process before - // getting killed. When this happens, we need to distinguish the duplicate blocks as they - // arrive. More details on this is explained in later comments. - - // Track if the block is received after shuffle merge finalize - final boolean isTooLate = partitionInfoBeforeCheck == null; - // Check if the given block is already merged by checking the bitmap against the given mapId - final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null - && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck; - - return new StreamCallbackWithID() { - private int length = 0; - private boolean canWrite = true; - // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly - private List deferredBufs; - - @Override - public String getID() { - return msg.blockId; - } - - /** - * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of - * the block data written to file. In case of failure during writing block to file, - * we use the information tracked in partitionInfo to overwrite the corrupt block - * when writing the new block. - */ - private void writeBuf(ByteBuffer buf) throws IOException { - while (buf.hasRemaining()) { - if (partitionInfo.isEncounteredFailure()) { - length += partitionInfo.channel.write(buf, partitionInfo.getPosition() + length); - } else { - length += partitionInfo.channel.write(buf); - } - } - } - - @Override - public void onData(String streamId, ByteBuffer buf) throws IOException { - // If partition info is null, ignore the requests. It could only be - // null either when a request is received after the shuffle file is - // finalized or when a request is for a duplicate block. - if (partitionInfo == null) { - return; - } - // When handling the block data using StreamInterceptor, it can help to reduce the amount - // of data that needs to be buffered in memory since it does not wait till the completion - // of the frame before handling the message, thus releasing the ByteBuf earlier. However, - // this also means it would chunk a block into multiple buffers. Here, we want to preserve - // the benefit of handling the block data using StreamInterceptor as much as possible while - // providing the guarantee that one block would be continuously written to the merged - // shuffle file before the next block starts. For each shuffle partition, we would track - // the current map id to make sure only block matching the map id can be written to disk. - // If one server thread sees the block being handled is the current block, it would - // directly write the block to disk. Otherwise, it would buffer the block chunks in memory. - // If the block becomes the current block before we see the end of it, we would then dump - // all buffered block data to disk and write the remaining portions of the block directly - // to disk as well. This way, we avoid having to buffer the entirety of every blocks in - // memory, while still providing the necessary guarantee. - synchronized (partitionInfo) { - // If the key is no longer present in the map, it means the shuffle merge has already - // been finalized. We should thus ignore the data and just drain the remaining bytes of - // this message. This check should be placed inside the synchronized block to make sure - // that checking the key is still present and processing the data is atomic. - if (!partitions.containsKey(partitionId)) { - // TODO is it necessary to dereference deferredBufs? - deferredBufs = null; - return; - } - // Check whether we can write to disk - if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { - // Check if this is a duplicate block generated by speculative tasks. With speculative - // tasks, we could receive the same block from 2 different sources at the same time. - // One of them is going to be the first to set the currentMapId. When that block does - // so, it's going to see the currentMapId initially as -1. After it sets the - // currentMapId, it's going to write some data to disk, thus increasing the length - // counter. The other duplicate block is going to see the currentMapId already set to - // its mapId. However, it hasn't written any data yet. If the first block gets written - // completely and resets the currentMapId to -1 before the processing for the second - // block finishes, we can just check the bitmap to identify the second as a duplicate. - if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || - partitionInfo.mapTracker.contains(mapId)) { - deferredBufs = null; - return; - } - if (partitionInfo.getCurrentMapId() < 0) { - partitionInfo.setCurrentMapId(mapId); - } - - // If we got here, it's safe to write the block data to the merged shuffle file. We - // first write any deferred block chunk buffered in memory, then write the remaining - // of the block. - if (deferredBufs != null && !deferredBufs.isEmpty()) { - for (ByteBuffer deferredBuf : deferredBufs) { - writeBuf(deferredBuf); - } - deferredBufs = null; - } - writeBuf(buf); - // If we got here, it means we successfully write the current chunk of block to merged - // shuffle file. If we encountered failure while writing the previous block, we should - // reset the file channel position and the status of partitionInfo to indicate that we - // have recovered from previous disk write failure. However, we do not update the - // position tracked by partitionInfo here. That is only updated while the entire block - // is successfully written to merged shuffle file. - if (partitionInfo.isEncounteredFailure()) { - partitionInfo.channel.position(partitionInfo.getPosition() + length); - partitionInfo.setEncounteredFailure(false); - } - } else { - // If we cannot write to disk, we buffer the current block chunk in memory so it could - // potentially be written to disk later. We take our best effort without guarantee - // that the block will be written to disk. If the block data is divided into multiple - // chunks during TCP transportation, each #onData invocation is an attempt to write - // the block to disk. If the block is still not written to disk after all #onData - // invocations, the final #onComplete invocation is the last attempt to write the - // block to disk. If we still couldn't write this block to disk after this, we give up - // on this block push request and respond failure to client. We could potentially - // buffer the block longer or wait for a few iterations inside #onData or #onComplete - // to increase the chance of writing the block to disk, however this would incur more - // memory footprint or decrease the server processing throughput for the shuffle - // service. In addition, during test we observed that by randomizing the order in - // which clients sends block push requests batches, only ~0.5% blocks failed to be - // written to disk due to this reason. We thus decide to optimize for server - // throughput and memory usage. - if (deferredBufs == null) { - deferredBufs = new LinkedList<>(); - } - // Write the buffer to the in-memory deferred cache - ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining()); - deferredBuf.put(buf); - deferredBuf.flip(); - deferredBufs.add(deferredBuf); - } - } - } - - @Override - public void onComplete(String streamId) throws IOException { - if (partitionInfo == null) { - if (isTooLate) { - // Throw an exception here so the block data is drained from channel and server - // responds RpcFailure to the client. - throw new RuntimeException(String.format("Block %s %s", msg.blockId, - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); - } else { - // For duplicate block that is received before the shuffle merge finalizes, the - // server should respond success to the client. - return; - } - } - // TODO should the merge manager check for the merge completion ratio here and finalize - // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early - // TODO and the file channel can be closed even if finalize merge request is somehow not - // TODO received from the driver? If so, then we need to know # maps for this shuffle. - - synchronized (partitionInfo) { - // When this request initially got to the server, the shuffle merge finalize request - // was not received yet. By the time we finish reading this message, the shuffle merge - // however is already finalized. We should thus respond RpcFailure to the client. - if (!partitions.containsKey(partitionId)) { - deferredBufs = null; - throw new RuntimeException(String.format("Block %s %s", msg.blockId, - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); - } - // Check if we can commit this block - if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { - // Identify duplicate block generated by speculative tasks. We respond success to - // the client in cases of duplicate even though no data is written. - if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || - partitionInfo.mapTracker.contains(mapId)) { - deferredBufs = null; - return; - } - if (partitionInfo.getCurrentMapId() < 0 && deferredBufs != null - && !deferredBufs.isEmpty()) { - for (ByteBuffer deferredBuf : deferredBufs) { - writeBuf(deferredBuf); - } - deferredBufs = null; - } - long updatedPos = partitionInfo.getPosition() + length; - boolean indexUpdated = false; - if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { - partitionInfo.updateLastChunkOffset(updatedPos, mapId); - indexUpdated = true; - } - partitionInfo.setPosition(updatedPos); - partitionInfo.setCurrentMapId(-1); - - // update merged results - partitionInfo.blockMerged(mapId); - if (indexUpdated) { - partitionInfo.resetChunkTracker(); - } - } else { - deferredBufs = null; - canWrite = false; - throw new RuntimeException(String.format("%s %s to merged shuffle", - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX, msg.blockId)); - } - } - } - - @Override - public void onFailure(String streamId, Throwable throwable) throws IOException { - if ((throwable.getMessage() != null && - (throwable.getMessage().contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || - throwable.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) || - - (throwable.getCause() != null && throwable.getCause().getMessage() != null && - (throwable.getCause().getMessage().contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || - throwable.getCause().getMessage().contains( - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { - logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); - } else { - logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable); - } - // Only update partitionInfo if the failure corresponds to a valid request. If the - // request is too late, i.e. received after shuffle merge finalize, #onFailure will - // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity - // to write the block data to disk, we should also ignore here. - if (canWrite && partitionInfo != null && partitions.containsKey(partitionId)) { - synchronized (partitionInfo) { - partitionInfo.setCurrentMapId(-1); - partitionInfo.setEncounteredFailure(true); - } - } - } - }; - } - - @Override - public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { - logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId); - List bitmaps = new LinkedList<>(); - List reduceIds = new LinkedList<>(); - List sizes = new LinkedList<>(); - Iterator> iterator = - partitions.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - AppShufflePartitionId partitionId = entry.getKey(); - AppShufflePartitionInfo partition = entry.getValue(); - if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) { - synchronized (partition) { - iterator.remove(); - // Get rid of any partial block data at the end of the file. This could either - // be due to failure or a request still being processed when the shuffle - // merge gets finalized. - try { - partition.channel.truncate(partition.getPosition()); - if (partition.getPosition() != partition.getLastChunkOffset()) { - partition.updateLastChunkOffset(partition.getPosition(), partition.lastMergedMapId); - } - bitmaps.add(partition.mapTracker); - reduceIds.add(partitionId.reduceId); - sizes.add(partition.getPosition()); - } catch (IOException ioe) { - logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId, - msg.shuffleId, partitionId.reduceId, ioe); - } finally { - try { - partition.channel.close(); - partition.metaChannel.close(); - partition.indexWriteStream.close(); - } catch (IOException closeEx) { - logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId, - msg.shuffleId, partitionId.reduceId, closeEx); - } - } - } - } - } - logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId); - return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), - Ints.toArray(reduceIds), Longs.toArray(sizes)); - } - - @Override - public void registerApplication(String appId, String user) { - logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user); - appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user)); - } - - @Override - public void registerExecutor(String appId, String[] localDirs) { - if (logger.isDebugEnabled()) { - logger.debug("register executor with RemoteBlockPushResolver {} {}", - appId, Arrays.toString(localDirs)); - } - Preconditions.checkNotNull(appsPathInfo.get(appId), - "application " + appId + " is not registered or NM was restarted."); - appsPathInfo.compute(appId, - (targetAppId, appPathsInfo) -> - appPathsInfo.updateActiveLocalDirs( - targetAppId, relativeMergeDirPathPattern, localDirs)); - } - - /** - * ID that uniquely identifies a shuffle partition for an application. This is used to key - * the metadata tracked for each shuffle partition that's being actively merged. - */ - public static class AppShufflePartitionId { - public final String appId; - public final int shuffleId; - public final int reduceId; - - AppShufflePartitionId(String appId, int shuffleId, int reduceId) { - this.appId = appId; - this.shuffleId = shuffleId; - this.reduceId = reduceId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AppShufflePartitionId that = (AppShufflePartitionId) o; - return shuffleId == that.shuffleId && reduceId == that.reduceId - && Objects.equal(appId, that.appId); - } - - @Override - public int hashCode() { - return Objects.hashCode(appId, shuffleId, reduceId); - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("shuffleId", shuffleId) - .add("reduceId", reduceId) - .toString(); - } - - String generateFileName() { - return String.format("mergedShuffle_%s_%d_%d.data", appId, shuffleId, reduceId); - } - - String generateIndexFileName() { - return String.format("mergedShuffle_%s_%d_%d.index", appId, shuffleId, reduceId); - } - - String generateMetaFileName() { - return String.format("mergedShuffle_%s_%d_%d.meta", appId, shuffleId, reduceId); - } - - boolean compareAppShuffleId(String appId, int shuffleId) { - return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId; - } - } - - /** - * Metadata tracked for an actively merged shuffle partition - */ - public static class AppShufflePartitionInfo { - // The merged shuffle data file - final File targetFile; - public final FileChannel channel; - // Location offset of the last successfully merged block for this shuffle partition - private long position; - // Indicating whether failure was encountered when merging the previous block - private boolean encounteredFailure; - // Track the map Id whose block is being merged for this shuffle partition - private int currentMapId; - // Bitmap tracking which mapper's blocks have been merged for this shuffle partition - private RoaringBitmap mapTracker; - // The merged shuffle index file - private final FileChannel indexChannel; - private final FileChannel metaChannel; - private final DataOutputStream indexWriteStream; - // The offset for the last chunk tracked in the index file for this shuffle partition - private long lastChunkOffset; - private int lastMergedMapId; - - // Bitmap tracking which mapper's blocks are in shuffle chunk - private RoaringBitmap chunkTracker; - ByteBuf trackerBuf = null; - - AppShufflePartitionInfo(File targetFile, File indexFile, File metaFile) throws IOException { - targetFile.createNewFile(); - this.targetFile = targetFile; - this.channel = new FileOutputStream(targetFile, true).getChannel(); - indexFile.createNewFile(); - FileOutputStream fos = new FileOutputStream(indexFile, true); - indexChannel = fos.getChannel(); - this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); - metaFile.createNewFile(); - metaChannel = new FileOutputStream(metaFile, true).getChannel(); - this.currentMapId = -1; - // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() - updateLastChunkOffset(0L, -1); - this.position = 0; - this.encounteredFailure = false; - this.mapTracker = new RoaringBitmap(); - this.chunkTracker = new RoaringBitmap(); - } - - public long getPosition() { - return position; - } - - public void setPosition(long position) { - this.position = position; - } - - boolean isEncounteredFailure() { - return encounteredFailure; - } - - void setEncounteredFailure(boolean encounteredFailure) { - this.encounteredFailure = encounteredFailure; - } - - int getCurrentMapId() { - return currentMapId; - } - - void setCurrentMapId(int mapId) { - this.currentMapId = mapId; - } - - long getLastChunkOffset() { - return lastChunkOffset; - } - - void blockMerged(int mapId) { - mapTracker.add(mapId); - chunkTracker.add(mapId); - lastMergedMapId = mapId; - } - - void resetChunkTracker() { - chunkTracker.clear(); - } - - void updateLastChunkOffset(long lastChunkOffset, int mapId) throws IOException { - long idxStartPos = -1; - try { - // update the chunk tracker to meta file before index file - writeChunkTracker(mapId); - idxStartPos = indexChannel.position(); - logger.trace("{} updated index with offset {}", targetFile.getName(), lastChunkOffset); - indexWriteStream.writeLong(lastChunkOffset); - } catch (IOException ioe) { - if (idxStartPos != -1) { - // reset the position to avoid corrupting index files during exception. - logger.warn("{} reset index to position {}", targetFile.getName(), idxStartPos); - indexChannel.position(idxStartPos); - } - throw ioe; - } - this.lastChunkOffset = lastChunkOffset; - } - - private void writeChunkTracker(int mapId) throws IOException { - if (mapId == -1) { - return; - } - chunkTracker.add(mapId); - if (trackerBuf == null) { - trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker)); - } - Encoders.Bitmaps.encode(trackerBuf, chunkTracker); - long metaStartPos = metaChannel.position(); - try { - logger.trace("{} write chunk to meta file", targetFile.getName()); - metaChannel.write(trackerBuf.nioBuffer()); - } catch (IOException ioe) { - logger.warn("{} reset position of meta file to {}", targetFile.getName(), metaStartPos); - metaChannel.position(metaStartPos); - throw ioe; - } finally { - trackerBuf.clear(); - } - } - } - - /** - * Wraps all the information related to the merge_dir of an application. - */ - private static class AppPathsInfo { - - private final String user; - private String[] activeLocalDirs; - - AppPathsInfo(String user) { - this.user = Preconditions.checkNotNull(user, "user cannot be null"); - } - - private AppPathsInfo updateActiveLocalDirs( - String appId, String relativePathPattern, String[] localDirs) { - if (activeLocalDirs == null) { - String relativePath = String.format(relativePathPattern, user, appId); - activeLocalDirs = Arrays.stream(localDirs) - .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) - .toArray(String[]::new); - logger.info("Updated the active local dirs {} for application {}", - Arrays.toString(activeLocalDirs), appId); - } - return this; - } - } -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java deleted file mode 100644 index 349c150ef1455..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java +++ /dev/null @@ -1,86 +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.protocol; - -import io.netty.buffer.ByteBuf; - -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.spark.network.protocol.Encoders; - -/** Base class for fetch shuffle blocks and chunks. */ -public abstract class AbstractFetchShuffleBlocks extends BlockTransferMessage { - public final String appId; - public final String execId; - public final int shuffleId; - - protected AbstractFetchShuffleBlocks( - String appId, - String execId, - int shuffleId) { - this.appId = appId; - this.execId = execId; - this.shuffleId = shuffleId; - } - - public ToStringBuilder toStringHelper() { - return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) - .append("appId", appId) - .append("execId", execId) - .append("shuffleId", shuffleId); - } - - /** - * Returns number of blocks in the request. - */ - public abstract int getNumBlocks(); - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - AbstractFetchShuffleBlocks that = (AbstractFetchShuffleBlocks) o; - if (shuffleId != that.shuffleId) return false; - if (!appId.equals(that.appId)) return false; - if (!execId.equals(that.execId)) return false; - return true; - } - - @Override - public int hashCode() { - int result = appId.hashCode(); - result = 31 * result + execId.hashCode(); - result = 31 * result + shuffleId; - return result; - } - - @Override - public int encodedLength() { - return Encoders.Strings.encodedLength(appId) - + Encoders.Strings.encodedLength(execId) - + 4; /* encoded length of shuffleId */ - } - - @Override - public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - Encoders.Strings.encode(buf, execId); - buf.writeInt(shuffleId); - } -} 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 a55a6cf7ed939..7f5058124988f 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 @@ -48,8 +48,7 @@ public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11), - PUSH_BLOCK_STREAM(12), FINALIZE_SHUFFLE_MERGE(13), MERGE_STATUSES(14), - FETCH_SHUFFLE_BLOCK_CHUNKS(15); + PUSH_BLOCK_STREAM(12), FINALIZE_SHUFFLE_MERGE(13), MERGE_STATUSES(14); private final byte id; @@ -83,7 +82,6 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 12: return PushBlockStream.decode(buf); case 13: return FinalizeShuffleMerge.decode(buf); case 14: return MergeStatuses.decode(buf); - case 15: return FetchShuffleBlockChunks.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/FetchShuffleBlockChunks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java deleted file mode 100644 index 6e867a7e89803..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java +++ /dev/null @@ -1,123 +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.protocol; - -import java.util.Arrays; - -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.protocol.Encoders; - -// Needed by ScalaDoc. See SPARK-7726 -import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; - - -/** Request to read a set of block chunks. Returns {@link StreamHandle}. */ -public class FetchShuffleBlockChunks extends AbstractFetchShuffleBlocks { - // The length of reduceIds must equal to chunkIds.size(). - // The i-th int[] in chunkIds contains all the chunks for the i-th reduceId in reduceIds. - public final int[] reduceIds; - public final int[][] chunkIds; - - public FetchShuffleBlockChunks( - String appId, - String execId, - int shuffleId, - int[] reduceIds, - int[][] chunkIds) { - super(appId, execId, shuffleId); - this.reduceIds = reduceIds; - this.chunkIds = chunkIds; - assert(reduceIds.length == chunkIds.length); - } - - @Override - protected Type type() { return Type.FETCH_SHUFFLE_BLOCK_CHUNKS; } - - @Override - public String toString() { - return toStringHelper() - .append("reduceIds", Arrays.toString(reduceIds)) - .append("chunkIds", Arrays.deepToString(chunkIds)) - .toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - FetchShuffleBlockChunks that = (FetchShuffleBlockChunks) o; - if (!super.equals(that)) return false; - if (!Arrays.equals(reduceIds, that.reduceIds)) return false; - return Arrays.deepEquals(chunkIds, that.chunkIds); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + Arrays.hashCode(reduceIds); - result = 31 * result + Arrays.deepHashCode(chunkIds); - return result; - } - - @Override - public int encodedLength() { - int encodedLengthOfChunkIds = 0; - for (int[] ids: chunkIds) { - encodedLengthOfChunkIds += Encoders.IntArrays.encodedLength(ids); - } - return super.encodedLength() - + Encoders.IntArrays.encodedLength(reduceIds) - + 4 /* encoded length of chunkIds.size() */ - + encodedLengthOfChunkIds; - } - - @Override - public void encode(ByteBuf buf) { - super.encode(buf); - Encoders.IntArrays.encode(buf, reduceIds); - buf.writeInt(chunkIds.length); - for (int[] ids: chunkIds) { - Encoders.IntArrays.encode(buf, ids); - } - } - - @Override - public int getNumBlocks() { - int numBlocks = 0; - for (int[] ids : chunkIds) { - numBlocks += ids.length; - } - return numBlocks; - } - - public static FetchShuffleBlockChunks decode(ByteBuf buf) { - String appId = Encoders.Strings.decode(buf); - String execId = Encoders.Strings.decode(buf); - int shuffleId = buf.readInt(); - int[] reduceIds = Encoders.IntArrays.decode(buf); - int chunkIdsLen = buf.readInt(); - int[][] chunkIds = new int[chunkIdsLen][]; - for (int i = 0; i < chunkIdsLen; i++) { - chunkIds[i] = Encoders.IntArrays.decode(buf); - } - return new FetchShuffleBlockChunks( - appId, execId, shuffleId, reduceIds, chunkIds); - } -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index f1bf9927be0ee..98057d58f7ab5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -29,7 +29,10 @@ import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** Request to read a set of blocks. Returns {@link StreamHandle}. */ -public class FetchShuffleBlocks extends AbstractFetchShuffleBlocks { +public class FetchShuffleBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final int shuffleId; // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. public final long[] mapIds; @@ -47,7 +50,9 @@ public FetchShuffleBlocks( long[] mapIds, int[][] reduceIds, boolean batchFetchEnabled) { - super(appId, execId, shuffleId); + this.appId = appId; + this.execId = execId; + this.shuffleId = shuffleId; this.mapIds = mapIds; this.reduceIds = reduceIds; assert(mapIds.length == reduceIds.length); @@ -64,7 +69,10 @@ public FetchShuffleBlocks( @Override public String toString() { - return toStringHelper() + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("shuffleId", shuffleId) .append("mapIds", Arrays.toString(mapIds)) .append("reduceIds", Arrays.deepToString(reduceIds)) .append("batchFetchEnabled", batchFetchEnabled) @@ -77,40 +85,35 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; FetchShuffleBlocks that = (FetchShuffleBlocks) o; - if (!super.equals(that)) return false; + + if (shuffleId != that.shuffleId) return false; if (batchFetchEnabled != that.batchFetchEnabled) return false; + if (!appId.equals(that.appId)) return false; + if (!execId.equals(that.execId)) return false; if (!Arrays.equals(mapIds, that.mapIds)) return false; return Arrays.deepEquals(reduceIds, that.reduceIds); } @Override public int hashCode() { - int result = super.hashCode(); + int result = appId.hashCode(); + result = 31 * result + execId.hashCode(); + result = 31 * result + shuffleId; result = 31 * result + Arrays.hashCode(mapIds); result = 31 * result + Arrays.deepHashCode(reduceIds); result = 31 * result + (batchFetchEnabled ? 1 : 0); return result; } - @Override - public int getNumBlocks() { - if (batchFetchEnabled) { - return mapIds.length; - } - int numBlocks = 0; - for (int[] ids : reduceIds) { - numBlocks += ids.length; - } - return numBlocks; - } - @Override public int encodedLength() { int encodedLengthOfReduceIds = 0; for (int[] ids: reduceIds) { encodedLengthOfReduceIds += Encoders.IntArrays.encodedLength(ids); } - return super.encodedLength() + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + 4 /* encoded length of shuffleId */ + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds @@ -119,7 +122,9 @@ public int encodedLength() { @Override public void encode(ByteBuf buf) { - super.encode(buf); + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + buf.writeInt(shuffleId); Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java similarity index 54% rename from common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java index e3f6d252208a1..608e784efb6e1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocksSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java @@ -15,28 +15,27 @@ * limitations under the License. */ -package org.apache.spark.network.shuffle.protocol; +package org.apache.spark.network.shuffle; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import org.junit.Assert; +import java.io.IOException; + +import com.google.common.base.Throwables; import org.junit.Test; +import org.apache.spark.network.shuffle.protocol.PushBlockStream; +import org.apache.spark.network.util.JavaUtils; + import static org.junit.Assert.*; -public class FetchShuffleBlocksSuite { +public class BlockPushExceptionSuite { @Test - public void testFetchShuffleBlockEncodeDecode() { - FetchShuffleBlocks fetchShuffleBlocks = - new FetchShuffleBlocks("app0", "exec1", 0, new long[] {0}, new int[][] {{0, 1}}, false); - Assert.assertEquals(2, fetchShuffleBlocks.getNumBlocks()); - int len = fetchShuffleBlocks.encodedLength(); - Assert.assertEquals(50, len); - ByteBuf buf = Unpooled.buffer(len); - fetchShuffleBlocks.encode(buf); - - FetchShuffleBlocks decoded = FetchShuffleBlocks.decode(buf); - assertEquals(fetchShuffleBlocks, decoded); + public void testDecodeExceptionMsg() { + PushBlockStream header = new PushBlockStream("app_0", "block_0", 128); + IOException ioexp = new IOException("Test exception"); + String encodedMsg = JavaUtils.encodeHeaderIntoErrorString(header.toByteBuffer(), ioexp); + BlockPushException exp = BlockPushException.decodeException(encodedMsg); + assertEquals(header, exp.getHeader()); + assertEquals(Throwables.getStackTraceAsString(ioexp), exp.getMessage()); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 909ff1ef6bf4d..93593bf3a7ca7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -23,7 +23,6 @@ import com.codahale.metrics.Meter; import com.codahale.metrics.Timer; -import org.apache.spark.network.client.MergedBlockMetaResponseCallback; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -42,7 +41,6 @@ import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; -import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; import org.apache.spark.network.shuffle.protocol.MergeStatuses; import org.apache.spark.network.shuffle.protocol.OpenBlocks; @@ -74,10 +72,12 @@ public void beforeEach() { public void testRegisterExecutor() { RpcResponseCallback callback = mock(RpcResponseCallback.class); - ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); + String[] localDirs = new String[] {"/a", "/b"}; + ExecutorShuffleInfo config = new ExecutorShuffleInfo(localDirs, 16, "sort"); ByteBuffer registerMessage = new RegisterExecutor("app0", "exec1", config).toByteBuffer(); handler.receive(client, registerMessage, callback); verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); + verify(mergedShuffleManager, times(1)).registerExecutor("app0", localDirs); verify(callback, times(1)).onSuccess(any(ByteBuffer.class)); verify(callback, never()).onFailure(any(Throwable.class)); @@ -231,37 +231,6 @@ public void testBadMessages() { verify(callback, never()).onFailure(any(Throwable.class)); } - @Test - public void testFetchMergedBlocksMeta() { - when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 0)).thenReturn( - new MergedBlockMeta(1, mock(ManagedBuffer.class))); - when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 1)).thenReturn( - new MergedBlockMeta(3, mock(ManagedBuffer.class))); - when(mergedShuffleManager.getMergedBlockMeta("app0", 0, 2)).thenReturn( - new MergedBlockMeta(5, mock(ManagedBuffer.class))); - - int[] expectedCount = new int[]{1, 3, 5}; - String appId = "app0"; - - for (int reduceId = 0; reduceId < 3; reduceId++) { - String blockId = "shuffle_0_-1_" + reduceId; - MergedBlockMetaResponseCallback callback = mock(MergedBlockMetaResponseCallback.class); - - handler.getMergedBlockMetaReqHandler() - .receiveMergeBlockMetaReq(client, appId, blockId, callback); - verify(mergedShuffleManager, times(1)).getMergedBlockMeta("app0", 0, reduceId); - - ArgumentCaptor numChunksResponse = ArgumentCaptor.forClass(Integer.class); - ArgumentCaptor chunkBitmapResponse = - ArgumentCaptor.forClass(ManagedBuffer.class); - verify(callback, times(1)).onSuccess(numChunksResponse.capture(), - chunkBitmapResponse.capture()); - assertEquals("num chunks in merged block " + reduceId, expectedCount[reduceId], - numChunksResponse.getValue().intValue()); - assertNotNull("chunks bitmap buffer " + reduceId, chunkBitmapResponse.getValue()); - } - } - @Test public void testFinalizeShuffleMerge() throws IOException { RpcResponseCallback callback = mock(RpcResponseCallback.class); @@ -282,97 +251,11 @@ public void testFinalizeShuffleMerge() throws IOException { MergeStatuses mergeStatuses = (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); assertEquals(mergeStatuses, statuses); - } - @Test - public void testOpenBlocksWithShuffleChunks() { - verifyBlockChunkFetches(true); - } - - @Test - public void testFetchShuffleChunks() { - verifyBlockChunkFetches(false); - } - - private void verifyBlockChunkFetches(boolean useOpenBlocks) { - RpcResponseCallback callback = mock(RpcResponseCallback.class); - ByteBuffer buffer; - if (useOpenBlocks) { - OpenBlocks openBlocks = - new OpenBlocks( - "app0", - "exec1", - new String[] { - "shuffleChunk_0_0_0", - "shuffleChunk_0_0_1", - "shuffleChunk_0_1_0", - "shuffleChunk_0_1_1" - }); - buffer = openBlocks.toByteBuffer(); - } else { - FetchShuffleBlockChunks fetchChunks = - new FetchShuffleBlockChunks( - "app0", "exec1", 0, new int[] {0, 1}, new int[][] {{0, 1}, {0, 1}}); - buffer = fetchChunks.toByteBuffer(); - } - ManagedBuffer[][] buffers = - new ManagedBuffer[][] { - { - new NioManagedBuffer(ByteBuffer.wrap(new byte[5])), - new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) - }, - { - new NioManagedBuffer(ByteBuffer.wrap(new byte[5])), - new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) - } - }; - for (int reduceId = 0; reduceId < 2; reduceId++) { - for (int chunkId = 0; chunkId < 2; chunkId++) { - when(mergedShuffleManager.getMergedBlockData( - "app0", 0, reduceId, chunkId)).thenReturn(buffers[reduceId][chunkId]); - } - } - - handler.receive(client, buffer, callback); - - ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); - verify(callback, times(1)).onSuccess(response.capture()); - verify(callback, never()).onFailure(any()); - - StreamHandle handle = - (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); - assertEquals(4, handle.numChunks); - - @SuppressWarnings("unchecked") - ArgumentCaptor> stream = (ArgumentCaptor>) - (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); - verify(streamManager, times(1)).registerStream(any(), stream.capture(), any()); - - Iterator bufferIter = stream.getValue(); - for (int reduceId = 0; reduceId < 2; reduceId++) { - for (int chunkId = 0; chunkId < 2; chunkId++) { - assertEquals(buffers[reduceId][chunkId], bufferIter.next()); - } - } - assertFalse(bufferIter.hasNext()); - - verify(mergedShuffleManager, never()).getMergedBlockMeta(anyString(), anyInt(), anyInt()); - verify(blockResolver, never()).getBlockData( - anyString(), anyString(), anyInt(), anyInt(), anyInt()); - verify(mergedShuffleManager, times(1)).getMergedBlockData("app0", 0, 0, 0); - verify(mergedShuffleManager, times(1)).getMergedBlockData("app0", 0, 0, 1); - - // Verify open block request latency metrics - Timer openBlockRequestLatencyMillis = (Timer) ((ExternalBlockHandler) handler) + Timer finalizeShuffleMergeLatencyMillis = (Timer) ((ExternalBlockHandler) handler) .getAllMetrics() .getMetrics() - .get("openBlockRequestLatencyMillis"); - assertEquals(1, openBlockRequestLatencyMillis.getCount()); - // Verify block transfer metrics - Meter blockTransferRateBytes = (Meter) ((ExternalBlockHandler) handler) - .getAllMetrics() - .getMetrics() - .get("blockTransferRateBytes"); - assertEquals(24, blockTransferRateBytes.getCount()); + .get("finalizeShuffleMergeLatencyMillis"); + assertEquals(1, finalizeShuffleMergeLatencyMillis.getCount()); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 8e4277ec39b54..285eedb39c65c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -46,7 +46,6 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; -import org.apache.spark.network.shuffle.protocol.FetchShuffleBlockChunks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.MapConfigProvider; @@ -202,47 +201,6 @@ public void testEmptyBlockFetch() { } } - @Test - public void testShuffleBlockChunksFetch() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("shuffleChunk_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("shuffleChunk_0_0_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); - blocks.put("shuffleChunk_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); - String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - - BlockFetchingListener listener = fetchBlocks( - blocks, - blockIds, - new FetchShuffleBlockChunks("app-id", "exec-id", 0, - new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), - conf); - - for (int i = 0; i < 3; i ++) { - verify(listener, times(1)).onBlockFetchSuccess( - "shuffleChunk_0_0_" + i, blocks.get("shuffleChunk_0_0_" + i)); - } - } - - @Test - public void testShuffleBlockChunkFetchFailure() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("shuffleChunk_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("shuffleChunk_0_0_1", null); - blocks.put("shuffleChunk_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); - String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - - BlockFetchingListener listener = fetchBlocks(blocks, blockIds, - new FetchShuffleBlockChunks("app-id", "exec-id", 0, new int[]{0}, - new int[][]{{0, 1, 2}}), conf); - - verify(listener, times(1)).onBlockFetchSuccess( - "shuffleChunk_0_0_0", blocks.get("shuffleChunk_0_0_0")); - verify(listener, times(1)).onBlockFetchFailure( - eq("shuffleChunk_0_0_1"), any()); - verify(listener, times(1)).onBlockFetchSuccess( - "shuffleChunk_0_0_2", blocks.get("shuffleChunk_0_0_2")); - } - /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java deleted file mode 100644 index 4f7a56cd41da7..0000000000000 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ /dev/null @@ -1,253 +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; - -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Arrays; - -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; - -import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.roaringbitmap.RoaringBitmap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.junit.Assert.*; - -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; -import org.apache.spark.network.client.StreamCallbackWithID; -import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge; -import org.apache.spark.network.shuffle.protocol.PushBlockStream; -import org.apache.spark.network.util.MapConfigProvider; -import org.apache.spark.network.util.TransportConf; - -public class RemoteBlockPushResolverSuite { - - private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); - - private TransportConf conf; - private RemoteBlockPushResolver pushResolver; - private String[] localDirs; - private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/"; - private final String USER = "testuser"; - private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401"; - - @Before - public void before() throws IOException { - localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(), - Paths.get("target/l2").toAbsolutePath().toString()}; - cleanupLocalDirs(); - MapConfigProvider provider = new MapConfigProvider( - ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); - conf = new TransportConf("shuffle", provider); - pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH); - } - - @After - public void after() { - try { - cleanupLocalDirs(); - } catch (IOException e) { - // don't fail if clean up doesn't succeed. - log.warn("Error deleting test local dirs", e); - } - } - - private void cleanupLocalDirs() throws IOException { - for (String local : localDirs) { - FileUtils.deleteDirectory(new File(local)); - } - } - - @Test(expected = RuntimeException.class) - public void testNoIndexFile() { - try { - String appId = "app_NoIndexFile"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); - pushResolver.getMergedBlockMeta(appId, 0, 0); - removeApplication(appId); - } catch (Throwable t) { - assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); - Throwables.propagate(t); - } - } - - @Test - public void testChunkCountsAndBlockData() throws IOException { - String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); - PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), - }; - ByteBuffer[] blocks = new ByteBuffer[]{ - ByteBuffer.wrap(new byte[4]), - ByteBuffer.wrap(new byte[5]) - }; - pushBlockHelper(appId, pushBlocks, blocks); - MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}}); - removeApplication(appId); - } - - @Test - public void testMultipleBlocksInAChunk() throws IOException { - String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); - PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), - new PushBlockStream(appId, "shuffle_0_2_0", 0), - new PushBlockStream(appId, "shuffle_0_3_0", 0), - }; - ByteBuffer[] buffers = new ByteBuffer[]{ - ByteBuffer.wrap(new byte[2]), - ByteBuffer.wrap(new byte[3]), - ByteBuffer.wrap(new byte[5]), - ByteBuffer.wrap(new byte[3]) - }; - pushBlockHelper(appId, pushBlocks, buffers); - MergedBlockMeta meta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); - removeApplication(appId); - } - - @Test - public void testAppUsingFewerLocalDirs() throws IOException { - String appId = "app_AppUsingFewerLocalDirs"; - String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); - PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), - new PushBlockStream(appId, "shuffle_0_2_0", 0), - new PushBlockStream(appId, "shuffle_0_3_0", 0), - }; - ByteBuffer[] buffers = new ByteBuffer[]{ - ByteBuffer.wrap(new byte[2]), - ByteBuffer.wrap(new byte[3]), - ByteBuffer.wrap(new byte[5]), - ByteBuffer.wrap(new byte[3]) - }; - pushBlockHelper(appId, pushBlocks, buffers); - MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); - removeApplication(appId); - } - - @Test(expected = NullPointerException.class) - public void testUpdateLocalDirsOnlyOnce() throws IOException { - String appId = "app_App1"; - // First app init and executor register will store the active local dirs list - registerApplication(appId, USER); - String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); - // Any later app init or executor register from the same application - // won't change the active local dirs list - registerApplication(appId, USER); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); - activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); - removeApplication(appId); - try { - pushResolver.getMergedBlockDirs(appId); - } catch (Throwable e) { - assertTrue(e.getMessage() - .startsWith("application app_App1 is not registered or NM was restarted.")); - Throwables.propagate(e); - } - } - - /** - * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. - * This is because when the application gets removed, the directory cleaner removes the merged - * data and file in a different thread which can delete the relevant data of a different test. - */ - private void registerApplication(String appId, String user) throws IOException { - pushResolver.registerApplication(appId, user); - } - - private void registerExecutor(String appId, String[] localDirs) throws IOException { - pushResolver.registerExecutor(appId, localDirs); - for (String localDir : pushResolver.getMergedBlockDirs(appId)) { - Files.createDirectories(Paths.get(localDir)); - } - } - - private String[] prepareBlockManagerLocalDirs(String appId, String user, String[] localDirs){ - return Arrays.stream(localDirs) - .map(localDir -> - localDir + "/" + - String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, appId)) - .toArray(String[]::new); - } - - private void removeApplication(String appId) { - pushResolver.applicationRemoved(appId, true); - } - - private void validateChunks( - String appId, int shuffleId, int reduceId, MergedBlockMeta meta, - int[] expectedSizes, int[][] expectedMapsPerChunk) throws IOException { - assertEquals("num chunks", expectedSizes.length, meta.getNumChunks()); - RoaringBitmap[] bitmaps = meta.readChunkBitmaps(); - assertEquals("num of bitmaps", meta.getNumChunks(), bitmaps.length); - for (int i = 0; i < meta.getNumChunks(); i++) { - RoaringBitmap chunkBitmap = bitmaps[i]; - Arrays.stream(expectedMapsPerChunk[i]).forEach(x -> assertTrue(chunkBitmap.contains(x))); - } - for (int i = 0; i < meta.getNumChunks(); i++) { - FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); - assertEquals(expectedSizes[i], mb.getLength()); - } - } - - private void pushBlockHelper(String appId, PushBlockStream[] pushBlocks, ByteBuffer[] blocks) - throws IOException { - Preconditions.checkArgument(pushBlocks.length == blocks.length); - for (int i = 0; i < pushBlocks.length; i++) { - StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( - new PushBlockStream(appId, pushBlocks[i].blockId, 0)); - stream.onData(stream.getID(), blocks[i]); - stream.onComplete(stream.getID()); - } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); - } -} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java deleted file mode 100644 index a26c60844b5ae..0000000000000 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunksSuite.java +++ /dev/null @@ -1,42 +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.protocol; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import org.junit.Assert; -import org.junit.Test; - -import static org.junit.Assert.*; - -public class FetchShuffleBlockChunksSuite { - - @Test - public void testFetchShuffleBlockChunksEncodeDecode() { - FetchShuffleBlockChunks shuffleBlockChunks = - new FetchShuffleBlockChunks("app0", "exec1", 0, new int[] {0}, new int[][] {{0, 1}}); - Assert.assertEquals(2, shuffleBlockChunks.getNumBlocks()); - int len = shuffleBlockChunks.encodedLength(); - Assert.assertEquals(45, len); - ByteBuf buf = Unpooled.buffer(len); - shuffleBlockChunks.encode(buf); - - FetchShuffleBlockChunks decoded = FetchShuffleBlockChunks.decode(buf); - assertEquals(shuffleBlockChunks, decoded); - } -} diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 919458ac33d22..3d14318bf90f0 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -54,7 +54,6 @@ import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalBlockHandler; -import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.TransportConf; import org.apache.spark.network.yarn.util.HadoopConfigProvider; @@ -95,9 +94,6 @@ public class YarnShuffleService extends AuxiliaryService { static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure"; private static final boolean DEFAULT_STOP_ON_FAILURE = false; - // Used by shuffle merge manager to create merged shuffle files. - protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/"; - // just for testing when you want to find an open port @VisibleForTesting static int boundPort = -1; @@ -130,8 +126,6 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; - @VisibleForTesting - RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -178,8 +172,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); - blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); + blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests @@ -286,7 +279,6 @@ public void initializeApplication(ApplicationInitializationContext context) { } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } - shuffleMergeManager.registerApplication(appId, context.getUser()); } @Override @@ -305,9 +297,6 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); - // TODO change cleanupLocalDirs to false. These should be deleted by yarn when the app - // finishes. - shuffleMergeManager.applicationRemoved(appId, true); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala index 4ce46156c0a92..ea4d252f0dbae 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceMetricsSuite.scala @@ -62,8 +62,7 @@ class ExternalShuffleServiceMetricsSuite extends SparkFunSuite { "registerExecutorRequestLatencyMillis", "shuffle-server.usedDirectMemory", "shuffle-server.usedHeapMemory", - "finalizeShuffleMergeLatencyMillis", - "fetchMergedBlocksMetaLatencyMillis") + "finalizeShuffleMergeLatencyMillis") ) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala index 8bfc1a0d3ff58..9239d891aae3b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala @@ -40,8 +40,7 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { val allMetrics = Set( "openBlockRequestLatencyMillis", "registerExecutorRequestLatencyMillis", "blockTransferRateBytes", "registeredExecutorsSize", "numActiveConnections", - "numCaughtExceptions", "fetchMergedBlocksMetaLatencyMillis", - "finalizeShuffleMergeLatencyMillis") + "numCaughtExceptions", "finalizeShuffleMergeLatencyMillis") metrics.getMetrics.keySet().asScala should be (allMetrics) } From 90d632935abc98af315c613d0a18476334013a25 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Wed, 23 Sep 2020 12:42:24 -0700 Subject: [PATCH 15/29] Further prune changes that should go into a later PR. --- .../org/apache/spark/network/shuffle/ExternalBlockHandler.java | 3 --- .../spark/network/shuffle/ExternalShuffleBlockResolver.java | 1 - 2 files changed, 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index b8cb581ab8e07..6489fb9d8a8b1 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -192,9 +192,6 @@ protected void handleMessage( GetLocalDirsForExecutors msg = (GetLocalDirsForExecutors) msgObj; checkAuth(client, msg.appId); Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); - if (Arrays.stream(msg.execIds).anyMatch(String::isEmpty)) { - localDirs.put("", mergeManager.getMergedBlockDirs(msg.appId)); - } callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); } else if (msgObj instanceof FinalizeShuffleMerge) { 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 d0963d28b5d84..a6bcbb8850566 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 @@ -371,7 +371,6 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { public Map getLocalDirs(String appId, String[] execIds) { return Arrays.stream(execIds) - .filter(exec -> !exec.isEmpty()) .map(exec -> { ExecutorShuffleInfo info = executors.get(new AppExecId(appId, exec)); if (info == null) { From 2bdf8009f4d8bb4adfa54be66bbe8c59caea0923 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Wed, 23 Sep 2020 13:37:07 -0700 Subject: [PATCH 16/29] Fix review comments. --- .../spark/network/server/TransportRequestHandler.java | 6 +++--- .../java/org/apache/spark/network/shuffle/ErrorHandler.java | 2 +- .../apache/spark/network/shuffle/ExternalBlockHandler.java | 1 - .../spark/network/shuffle/BlockPushExceptionSuite.java | 4 ++-- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index e9dd32f1d181d..ad1e857676dbd 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -267,8 +267,8 @@ public String getID() { } catch (IOException ioe) { // No exception will be thrown here. req.meta.nioByteBuffer will not throw IOException // because it's a NettyManagedBuffer. This try-catch block is to make compiler happy. - logger.error("Error in handling failure while invoking RpcHandler#receive() on RPC id " - + req.requestId, e); + logger.error("Error in handling failure while invoking RpcHandler#receive() on RPC id {}", + req.requestId, e); } finally { req.meta.release(); } @@ -296,7 +296,7 @@ private void processOneWayMessage(OneWayMessage req) { /** * Make a full copy of a nio ByteBuffer. */ - private ByteBuffer cloneBuffer(ByteBuffer buf) { + private static ByteBuffer cloneBuffer(ByteBuffer buf) { ByteBuffer clone = ByteBuffer.allocate(buf.capacity()); clone.put(buf.duplicate()); clone.flip(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java index 805164193c72f..f73b9882fc8b7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -24,7 +24,7 @@ * and logged. * Note: {@link RetryingBlockFetcher} will delegate the exception to this handler only when * - remaining retries < max retries - * - exception is either an IOException or SaslException + * - exception is an IOException */ public interface ErrorHandler { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 6489fb9d8a8b1..15ff9c359c3c5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java index 608e784efb6e1..801229a509ea1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java @@ -22,11 +22,11 @@ import com.google.common.base.Throwables; import org.junit.Test; +import static org.junit.Assert.*; + import org.apache.spark.network.shuffle.protocol.PushBlockStream; import org.apache.spark.network.util.JavaUtils; -import static org.junit.Assert.*; - public class BlockPushExceptionSuite { @Test From 3e9e9e1fe0da1383e0a26bd8610032b18a94cf1d Mon Sep 17 00:00:00 2001 From: Min Shen Date: Wed, 23 Sep 2020 14:19:24 -0700 Subject: [PATCH 17/29] Fix unit test failure. --- .../org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 46e596575533d..a6a302ad5df95 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -405,6 +405,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd "openBlockRequestLatencyMillis", "registeredExecutorsSize", "registerExecutorRequestLatencyMillis", + "finalizeShuffleMergeLatencyMillis", "shuffle-server.usedDirectMemory", "shuffle-server.usedHeapMemory" )) From eb366d48baf8ac8079f1b3465e3870000d20f96e Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 25 Sep 2020 12:49:02 -0700 Subject: [PATCH 18/29] Address review comments. --- .../spark/network/protocol/Encoders.java | 57 +++++-------------- .../server/TransportRequestHandler.java | 6 +- .../spark/network/shuffle/ErrorHandler.java | 22 +++---- .../shuffle/ExternalBlockStoreClient.java | 1 - .../protocol/FinalizeShuffleMerge.java | 4 +- .../shuffle/protocol/MergeStatuses.java | 2 +- .../shuffle/protocol/PushBlockStream.java | 1 + 7 files changed, 34 insertions(+), 59 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 437d03362117f..58ece9815c1a0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -54,61 +54,32 @@ public static String decode(ByteBuf buf) { /** Bitmaps are encoded with their serialization length followed by the serialization bytes. */ public static class Bitmaps { public static int encodedLength(RoaringBitmap b) { - // Compress the bitmap before serializing it + // Compress the bitmap before serializing it. Note that since BlockTransferMessage + // needs to invoke encodedLength first to figure out the length for the ByteBuf, it + // guarantees that the bitmap will always be compressed before being serialized. b.trim(); b.runOptimize(); return 4 + b.serializedSizeInBytes(); } public static void encode(ByteBuf buf, RoaringBitmap b) { - ByteBuffer outBuffer = ByteBuffer.allocate(b.serializedSizeInBytes()); - try { - b.serialize(new DataOutputStream(new OutputStream() { - ByteBuffer buffer; - - OutputStream init(ByteBuffer buffer) { - this.buffer = buffer; - return this; - } - - @Override - public void close() { - } - - @Override - public void flush() { - } - - @Override - public void write(int b) { - buffer.put((byte) b); - } - - @Override - public void write(byte[] b) { - buffer.put(b); - } - - @Override - public void write(byte[] b, int off, int l) { - buffer.put(b, off, l); - } - }.init(outBuffer))); - } catch (IOException e) { - throw new RuntimeException("Exception while encoding bitmap", e); - } - byte[] bytes = outBuffer.array(); - buf.writeInt(bytes.length); - buf.writeBytes(bytes); + int encodedLength = b.serializedSizeInBytes(); + buf.writeInt(encodedLength); + // RoaringBitmap requires nio ByteBuffer for serde. We expose the netty ByteBuf as a nio + // ByteBuffer. Here, we need to explicitly manage the index so we can write into the + // ByteBuffer, and the write is reflected in the underneath ByteBuf. + b.serialize(buf.nioBuffer(buf.writerIndex(), encodedLength)); + buf.writerIndex(buf.writerIndex() + encodedLength); } public static RoaringBitmap decode(ByteBuf buf) { int length = buf.readInt(); - byte[] bytes = new byte[length]; - buf.readBytes(bytes); RoaringBitmap bitmap = new RoaringBitmap(); try { - bitmap.deserialize(new DataInputStream(new ByteArrayInputStream(bytes))); + bitmap.deserialize(buf.nioBuffer()); + // RoaringBitmap deserialize does not advance the reader index of the underlying ByteBuf. + // Manually update the index here. + buf.readerIndex(buf.readerIndex() + length); } catch (IOException e) { throw new RuntimeException("Exception while decoding bitmap", e); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index ad1e857676dbd..c54a472f584ce 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -183,7 +183,9 @@ private void processStreamUpload(final UploadStream req) { assert (req.body() == null); try { // Retain the original metadata buffer, since it will be used during the invocation of - // this method. Will be released later. + // this method. Will be released later. The metadata will be sent back to the client + // in the response in case the client needs the original metadata to handle the response + // in a callback. req.meta.retain(); // Make a copy of the original metadata buffer. In benchmark, we noticed that // we cannot respond the original metadata buffer back to the client, otherwise @@ -269,6 +271,8 @@ public String getID() { // because it's a NettyManagedBuffer. This try-catch block is to make compiler happy. logger.error("Error in handling failure while invoking RpcHandler#receive() on RPC id {}", req.requestId, e); + assert false : "Unexpected IOException in handling failure while invoking " + + "RpcHandler#receive()"; } finally { req.meta.release(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java index f73b9882fc8b7..2da776236e2e8 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -51,21 +51,21 @@ public boolean shouldRetryError(Throwable t) { return false; } // If the block is too late, there is no need to retry it - return (t.getMessage() == null || !t.getMessage() - .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) && (t.getCause() == null - || t.getCause().getMessage() == null || !t.getCause().getMessage() - .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + return (t.getMessage() == null || + !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) && + (t.getCause() == null || t.getCause().getMessage() == null || + !t.getCause().getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); } @Override public boolean shouldLogError(Throwable t) { - return (t.getMessage() == null || ( - !t.getMessage().contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) - && !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) && ( - t.getCause() == null || t.getCause().getMessage() == null || (!t.getCause() - .getMessage() - .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && !t.getCause() - .getMessage() + return (t.getMessage() == null || + (!t.getMessage().contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) && + (t.getCause() == null || t.getCause().getMessage() == null || + (!t.getCause().getMessage() + .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + !t.getCause().getMessage() .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index 306d27fa6fe00..b7678baef5606 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -48,7 +48,6 @@ * (via BlockTransferService), which has the downside of losing the data if we lose the executors. */ public class ExternalBlockStoreClient extends BlockStoreClient { - private static final Logger logger = LoggerFactory.getLogger(ExternalBlockStoreClient.class); private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); private final TransportConf conf; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java index 8c9dd8aebba17..563ec60cede25 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java @@ -22,9 +22,9 @@ import org.apache.spark.network.protocol.Encoders; - /** - * Request to notify external shuffle service to finalize merge for a given shuffle. + * Request from the DAGScheduler to notify external shuffle service to finalize merge + * for a given shuffle. * Returns {@link MergeStatuses} */ public class FinalizeShuffleMerge extends BlockTransferMessage { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java index ab16e9450f874..7ba059696d40b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java @@ -27,7 +27,7 @@ /** - * Result returned by an ExternalShuffleService to a scheduler. This represents the result + * Result returned by an ExternalShuffleService to the DAGScheduler. This represents the result * of all the remote shuffle block merge operations performed by an ExternalShuffleService * for a given shuffle ID. It includes the shuffle ID, an array of bitmaps each representing * the set of mapper partition blocks that are merged for a given reducer partition, an array diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index 083b17aebe179..445ada5a1e2ed 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -28,6 +28,7 @@ /** * Request to push a block to a remote shuffle service to be merged in push based shuffle. + * The remote shuffle service will also include this message when responding the push requests. */ public class PushBlockStream extends BlockTransferMessage { public final String appId; From 7a6ab15a95f8b1ced0991654f5bc95957e658489 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 25 Sep 2020 12:53:45 -0700 Subject: [PATCH 19/29] Add more comment. --- .../apache/spark/network/shuffle/protocol/PushBlockStream.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index 445ada5a1e2ed..343e2312b979a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -33,6 +33,8 @@ public class PushBlockStream extends BlockTransferMessage { public final String appId; public final String blockId; + // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of + // blocks to be pushed. public final int index; public PushBlockStream(String appId, String blockId, int index) { From 85b0de8f48c8f998a41e794cf0a32c8bea35f237 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 25 Sep 2020 16:08:59 -0700 Subject: [PATCH 20/29] Fix style issue. --- .../java/org/apache/spark/network/protocol/Encoders.java | 5 ----- .../spark/network/shuffle/ExternalBlockStoreClient.java | 2 -- 2 files changed, 7 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 58ece9815c1a0..0256ef8d6dc9b 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -17,12 +17,7 @@ package org.apache.spark.network.protocol; -import java.io.ByteArrayInputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import io.netty.buffer.ByteBuf; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index b7678baef5606..ac5cdd8f864e0 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -28,8 +28,6 @@ import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; From 7b48c50b88f43a548bb845ca994b9e8794ef593e Mon Sep 17 00:00:00 2001 From: Min Shen Date: Tue, 29 Sep 2020 15:33:19 -0700 Subject: [PATCH 21/29] Add more clarifying comments. --- .../spark/network/shuffle/BlockPushException.java | 10 +++++++++- .../network/shuffle/MergedShuffleFileManager.java | 4 ++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java index 1c8e145d1eb2b..5a2cefec5333c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java @@ -34,11 +34,19 @@ public class BlockPushException extends RuntimeException { /** * String constant used for generating exception messages indicating a block to be merged * arrives too late on the server side, and also for later checking such exceptions on the - * client side. + * client side. When we get a block push failure because of the block arrives too late, we + * will not retry pushing the block nor log the exception on the client side. */ public static final String TOO_LATE_MESSAGE_SUFFIX = "received after merged shuffle is finalized"; + /** + * String constant used for generating exception messages indicating the server couldn't + * append a block after all available attempts due to collision with other blocks belonging + * to the same shuffle partition, and also for later checking such exceptions on the client + * side. When we get a block push failure because of the block couldn't be written due to + * this reason, we will not log the exception on the client side. + */ public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = "Couldn't find an opportunity to write block"; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java index da96ef4cae67d..ef4dbb2bd0059 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -31,8 +31,8 @@ * along side {@link ExternalBlockHandler} and serves as an RPCHandler for * {@link org.apache.spark.network.server.RpcHandler#receiveStream}, where it processes the * remotely pushed streams of shuffle blocks to merge them into merged shuffle files. Right - * now, push based shuffle can only be enabled when external shuffle service in YARN mode - * is used. + * now, support for push based shuffle is only implemented for external shuffle service in + * YARN mode. */ public interface MergedShuffleFileManager { /** From 9f00cc3a08044ea6c94abefa594a7d9610614df8 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Thu, 1 Oct 2020 10:36:22 -0700 Subject: [PATCH 22/29] Further optimize the serde of bitmap --- .../java/org/apache/spark/network/protocol/Encoders.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 0256ef8d6dc9b..4fa191b3917e3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -54,12 +54,11 @@ public static int encodedLength(RoaringBitmap b) { // guarantees that the bitmap will always be compressed before being serialized. b.trim(); b.runOptimize(); - return 4 + b.serializedSizeInBytes(); + return b.serializedSizeInBytes(); } public static void encode(ByteBuf buf, RoaringBitmap b) { int encodedLength = b.serializedSizeInBytes(); - buf.writeInt(encodedLength); // RoaringBitmap requires nio ByteBuffer for serde. We expose the netty ByteBuf as a nio // ByteBuffer. Here, we need to explicitly manage the index so we can write into the // ByteBuffer, and the write is reflected in the underneath ByteBuf. @@ -68,13 +67,12 @@ public static void encode(ByteBuf buf, RoaringBitmap b) { } public static RoaringBitmap decode(ByteBuf buf) { - int length = buf.readInt(); RoaringBitmap bitmap = new RoaringBitmap(); try { bitmap.deserialize(buf.nioBuffer()); // RoaringBitmap deserialize does not advance the reader index of the underlying ByteBuf. // Manually update the index here. - buf.readerIndex(buf.readerIndex() + length); + buf.readerIndex(buf.readerIndex() + bitmap.serializedSizeInBytes()); } catch (IOException e) { throw new RuntimeException("Exception while decoding bitmap", e); } From db36f3fcaab6793379f6fa99ee7d27f9b5abb90d Mon Sep 17 00:00:00 2001 From: Min Shen Date: Fri, 2 Oct 2020 15:04:57 -0700 Subject: [PATCH 23/29] Update common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java Co-authored-by: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com> --- .../org/apache/spark/network/shuffle/ErrorHandlerSuite.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java index 905a90d9c4cb0..29b76d6392e5e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -42,10 +42,10 @@ public void testPushErrorRetry() { @Test public void testPushErrorLogging() { ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); - assert (!handler.shouldLogError(new RuntimeException( + assertFalse(handler.shouldLogError(new RuntimeException( new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))); - assert (!handler.shouldLogError(new RuntimeException( + assertFalse(handler.shouldLogError(new RuntimeException( new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); - assert (handler.shouldLogError(new Throwable())); + assertTrue(handler.shouldLogError(new Throwable())); } } From e604686dd769c9111e29509568fc784e14aed485 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Sat, 10 Oct 2020 05:01:33 -0700 Subject: [PATCH 24/29] Address additional review comments --- .../server/TransportRequestHandler.java | 57 ++---------- .../apache/spark/network/util/JavaUtils.java | 12 --- .../spark/network/RpcIntegrationSuite.java | 37 +------- .../network/shuffle/BlockPushException.java | 86 ------------------- .../spark/network/shuffle/ErrorHandler.java | 32 +++++-- .../network/shuffle/OneForOneBlockPusher.java | 55 ++++++------ .../network/shuffle/RetryingBlockFetcher.java | 4 + .../shuffle/BlockPushExceptionSuite.java | 41 --------- .../network/shuffle/ErrorHandlerSuite.java | 16 ++-- .../shuffle/OneForOneBlockPusherSuite.java | 23 +++-- 10 files changed, 86 insertions(+), 277 deletions(-) delete mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java delete mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index c54a472f584ce..2be236ebd6815 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -32,7 +32,6 @@ import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.*; import org.apache.spark.network.protocol.*; -import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.TransportFrameDecoder; import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; @@ -182,19 +181,6 @@ public void onFailure(Throwable e) { private void processStreamUpload(final UploadStream req) { assert (req.body() == null); try { - // Retain the original metadata buffer, since it will be used during the invocation of - // this method. Will be released later. The metadata will be sent back to the client - // in the response in case the client needs the original metadata to handle the response - // in a callback. - req.meta.retain(); - // Make a copy of the original metadata buffer. In benchmark, we noticed that - // we cannot respond the original metadata buffer back to the client, otherwise - // in cases where multiple concurrent shuffles are present, a wrong metadata might - // be sent back to client. This is related to the eager release of the metadata buffer, - // i.e., we always release the original buffer by the time the invocation of this - // method ends, instead of by the time we respond it to the client. This is necessary, - // otherwise we start seeing memory issues very quickly in benchmarks. - ByteBuffer meta = cloneBuffer(req.meta.nioByteBuffer()); RpcResponseCallback callback = new RpcResponseCallback() { @Override public void onSuccess(ByteBuffer response) { @@ -203,17 +189,13 @@ public void onSuccess(ByteBuffer response) { @Override public void onFailure(Throwable e) { - // Piggyback request metadata as part of the exception error String, so we can - // respond the metadata upon a failure without changing the existing protocol. - respond(new RpcFailure(req.requestId, - JavaUtils.encodeHeaderIntoErrorString(meta.duplicate(), e))); - req.meta.release(); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); } }; TransportFrameDecoder frameDecoder = (TransportFrameDecoder) channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); - StreamCallbackWithID streamHandler = - rpcHandler.receiveStream(reverseClient, meta.duplicate(), callback); + ByteBuffer meta = req.meta.nioByteBuffer(); + StreamCallbackWithID streamHandler = rpcHandler.receiveStream(reverseClient, meta, callback); if (streamHandler == null) { throw new NullPointerException("rpcHandler returned a null streamHandler"); } @@ -227,17 +209,12 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { public void onComplete(String streamId) throws IOException { try { streamHandler.onComplete(streamId); - callback.onSuccess(meta.duplicate()); + callback.onSuccess(ByteBuffer.allocate(0)); } catch (Exception ex) { IOException ioExc = new IOException("Failure post-processing complete stream;" + " failing this rpc and leaving channel active", ex); - // req.meta will be released once inside callback.onFailure. Retain it one more - // time to be released in the finally block. - req.meta.retain(); callback.onFailure(ioExc); streamHandler.onFailure(streamId, ioExc); - } finally { - req.meta.release(); } } @@ -261,21 +238,7 @@ public String getID() { } } catch (Exception e) { logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); - try { - // It's OK to respond the original metadata buffer here, because this is still inside - // the invocation of this method. - respond(new RpcFailure(req.requestId, - JavaUtils.encodeHeaderIntoErrorString(req.meta.nioByteBuffer(), e))); - } catch (IOException ioe) { - // No exception will be thrown here. req.meta.nioByteBuffer will not throw IOException - // because it's a NettyManagedBuffer. This try-catch block is to make compiler happy. - logger.error("Error in handling failure while invoking RpcHandler#receive() on RPC id {}", - req.requestId, e); - assert false : "Unexpected IOException in handling failure while invoking " - + "RpcHandler#receive()"; - } finally { - req.meta.release(); - } + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); // We choose to totally fail the channel, rather than trying to recover as we do in other // cases. We don't know how many bytes of the stream the client has already sent for the // stream, it's not worth trying to recover. @@ -297,16 +260,6 @@ private void processOneWayMessage(OneWayMessage req) { } } - /** - * Make a full copy of a nio ByteBuffer. - */ - private static ByteBuffer cloneBuffer(ByteBuffer buf) { - ByteBuffer clone = ByteBuffer.allocate(buf.capacity()); - clone.put(buf.duplicate()); - clone.flip(); - return clone; - } - /** * Responds to a single message with some Encodable object. If a failure occurs while sending, * it will be logged and the channel closed. diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 956e937578bea..b5497087634ce 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -27,7 +27,6 @@ import java.util.regex.Pattern; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.apache.commons.lang3.SystemUtils; @@ -81,17 +80,6 @@ public static String bytesToString(ByteBuffer b) { return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); } - /** - * Encode the header ByteBuffer into the error string to be returned via RpcFailure. - * Use ISO_8859_1 encoding instead of UTF_8. UTF_8 will change the byte content - * for bytes larger than 127. This would render incorrect result when encoding - * decoding the index inside the PushBlockStream message. - */ - public static String encodeHeaderIntoErrorString(ByteBuffer headerBuffer, Throwable e) { - String encodedHeader = StandardCharsets.ISO_8859_1.decode(headerBuffer).toString(); - return encodedHeader + Throwables.getStackTraceAsString(e); - } - /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 7288060bd115b..916c140621671 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -19,7 +19,6 @@ import java.io.*; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; @@ -169,7 +168,6 @@ public static void tearDown() { static class RpcResult { public Set successMessages; public Set errorMessages; - public Set successResponses; } private RpcResult sendRPC(String ... commands) throws Exception { @@ -212,7 +210,6 @@ private RpcResult sendRpcWithStream(String... streams) throws Exception { RpcResult res = new RpcResult(); res.successMessages = Collections.synchronizedSet(new HashSet<>()); res.errorMessages = Collections.synchronizedSet(new HashSet<>()); - res.successResponses = Collections.synchronizedSet(new HashSet<>()); for (String stream : streams) { int idx = stream.lastIndexOf('/'); @@ -250,7 +247,6 @@ private static class RpcStreamCallback implements RpcResponseCallback { @Override public void onSuccess(ByteBuffer message) { res.successMessages.add(streamId); - res.successResponses.add(JavaUtils.bytesToString(message)); sem.release(); } @@ -330,7 +326,6 @@ public void sendRpcWithStreamOneAtATime() throws Exception { RpcResult res = sendRpcWithStream(stream); assertTrue("there were error messages!" + res.errorMessages, res.errorMessages.isEmpty()); assertEquals(Sets.newHashSet(stream), res.successMessages); - assertEquals(Sets.newHashSet(stream), res.successResponses); } } @@ -341,9 +336,7 @@ public void sendRpcWithStreamConcurrently() throws Exception { streams[i] = StreamTestHelper.STREAMS[i % StreamTestHelper.STREAMS.length]; } RpcResult res = sendRpcWithStream(streams); - Set streamSet = Sets.newHashSet(StreamTestHelper.STREAMS); - assertEquals(streamSet, res.successMessages); - assertEquals(streamSet, res.successResponses); + assertEquals(Sets.newHashSet(StreamTestHelper.STREAMS), res.successMessages); assertTrue(res.errorMessages.isEmpty()); } @@ -351,44 +344,22 @@ public void sendRpcWithStreamConcurrently() throws Exception { public void sendRpcWithStreamFailures() throws Exception { // when there is a failure reading stream data, we don't try to keep the channel usable, // just send back a decent error msg. - String failStream = "fail/exception-ondata/smallBuffer"; RpcResult exceptionInCallbackResult = - sendRpcWithStream(failStream, "smallBuffer"); + sendRpcWithStream("fail/exception-ondata/smallBuffer", "smallBuffer"); assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); - assertDecodedErrorsContain(exceptionInCallbackResult.errorMessages, failStream); - failStream = "fail/null/smallBuffer"; RpcResult nullStreamHandler = - sendRpcWithStream(failStream, "smallBuffer"); + sendRpcWithStream("fail/null/smallBuffer", "smallBuffer"); assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); - assertDecodedErrorsContain(nullStreamHandler.errorMessages, failStream); // OTOH, if there is a failure during onComplete, the channel should still be fine - failStream = "fail/exception-oncomplete/smallBuffer"; RpcResult exceptionInOnComplete = - sendRpcWithStream(failStream, "smallBuffer"); + sendRpcWithStream("fail/exception-oncomplete/smallBuffer", "smallBuffer"); assertErrorsContain(exceptionInOnComplete.errorMessages, Sets.newHashSet("Failure post-processing")); - assertDecodedErrorsContain(exceptionInOnComplete.errorMessages, failStream); assertEquals(Sets.newHashSet("smallBuffer"), exceptionInOnComplete.successMessages); } - private void assertDecodedErrorsContain(Set errors, String contain) { - Set decodedErrors = Sets.newHashSet(); - for (String error : errors) { - ByteBuffer rawBuffer = ByteBuffer.wrap(error.getBytes(StandardCharsets.ISO_8859_1)); - decodedErrors.add(JavaUtils.bytesToString(rawBuffer)); - } - boolean foundMatch = false; - for (String error : decodedErrors) { - if (error.contains(contain)) { - foundMatch = true; - break; - } - } - assertTrue("Could not find decoded error containing " + contain, foundMatch); - } - private void assertErrorsContain(Set errors, Set contains) { assertEquals("Expected " + contains.size() + " errors, got " + errors.size() + "errors: " + errors, contains.size(), errors.size()); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java deleted file mode 100644 index 5a2cefec5333c..0000000000000 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockPushException.java +++ /dev/null @@ -1,86 +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; - -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; - -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.PushBlockStream; - -/** - * A special exception type that would decode the encoded {@link PushBlockStream} from the - * exception String. This complements the encoding logic in - * {@link org.apache.spark.network.server.TransportRequestHandler}. - */ -public class BlockPushException extends RuntimeException { - private PushBlockStream header; - - /** - * String constant used for generating exception messages indicating a block to be merged - * arrives too late on the server side, and also for later checking such exceptions on the - * client side. When we get a block push failure because of the block arrives too late, we - * will not retry pushing the block nor log the exception on the client side. - */ - public static final String TOO_LATE_MESSAGE_SUFFIX = - "received after merged shuffle is finalized"; - - /** - * String constant used for generating exception messages indicating the server couldn't - * append a block after all available attempts due to collision with other blocks belonging - * to the same shuffle partition, and also for later checking such exceptions on the client - * side. When we get a block push failure because of the block couldn't be written due to - * this reason, we will not log the exception on the client side. - */ - public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = - "Couldn't find an opportunity to write block"; - - private BlockPushException(PushBlockStream header, String message) { - super(message); - this.header = header; - } - - public static BlockPushException decodeException(String message) { - // Use ISO_8859_1 encoding instead of UTF_8. UTF_8 will change the byte content - // for bytes larger than 127. This would render incorrect result when encoding - // decoding the index inside the PushBlockStream message. - ByteBuffer rawBuffer = ByteBuffer.wrap(message.getBytes(StandardCharsets.ISO_8859_1)); - try { - BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(rawBuffer); - if (msgObj instanceof PushBlockStream) { - PushBlockStream header = (PushBlockStream) msgObj; - // When decoding the header, the rawBuffer's position is not updated since it was - // consumed via netty's ByteBuf. Updating the rawBuffer's position here to retrieve - // the remaining exception message. - ByteBuffer remainingBuffer = (ByteBuffer) rawBuffer.position(rawBuffer.position() - + header.encodedLength() + 1); - return new BlockPushException(header, - StandardCharsets.UTF_8.decode(remainingBuffer).toString()); - } else { - throw new UnsupportedOperationException(String.format("Cannot decode the header. " - + "Expected PushBlockStream but got %s instead", msgObj.getClass().getSimpleName())); - } - } catch (Exception e) { - return new BlockPushException(null, message); - } - } - - public PushBlockStream getHeader() { - return header; - } -} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java index 2da776236e2e8..993d01cfc49be 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -28,6 +28,7 @@ */ public interface ErrorHandler { + boolean shouldRetryError(Throwable t); default boolean shouldLogError(Throwable t) { @@ -43,6 +44,25 @@ default boolean shouldLogError(Throwable t) { * The error handler for pushing shuffle blocks to remote shuffle services. */ class BlockPushErrorHandler implements ErrorHandler { + /** + * String constant used for generating exception messages indicating a block to be merged + * arrives too late on the server side, and also for later checking such exceptions on the + * client side. When we get a block push failure because of the block arrives too late, we + * will not retry pushing the block nor log the exception on the client side. + */ + public static final String TOO_LATE_MESSAGE_SUFFIX = + "received after merged shuffle is finalized"; + + /** + * String constant used for generating exception messages indicating the server couldn't + * append a block after all available attempts due to collision with other blocks belonging + * to the same shuffle partition, and also for later checking such exceptions on the client + * side. When we get a block push failure because of the block couldn't be written due to + * this reason, we will not log the exception on the client side. + */ + public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = + "Couldn't find an opportunity to write block"; + @Override public boolean shouldRetryError(Throwable t) { @@ -52,21 +72,21 @@ public boolean shouldRetryError(Throwable t) { } // If the block is too late, there is no need to retry it return (t.getMessage() == null || - !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)) && + !t.getMessage().contains(TOO_LATE_MESSAGE_SUFFIX)) && (t.getCause() == null || t.getCause().getMessage() == null || - !t.getCause().getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + !t.getCause().getMessage().contains(TOO_LATE_MESSAGE_SUFFIX)); } @Override public boolean shouldLogError(Throwable t) { return (t.getMessage() == null || - (!t.getMessage().contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && - !t.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) && + (!t.getMessage().contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + !t.getMessage().contains(TOO_LATE_MESSAGE_SUFFIX))) && (t.getCause() == null || t.getCause().getMessage() == null || (!t.getCause().getMessage() - .contains(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + .contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && !t.getCause().getMessage() - .contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))); + .contains(TOO_LATE_MESSAGE_SUFFIX))); } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java index feee7ba5c0279..3fc0eb1f367ec 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -28,7 +28,6 @@ import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.PushBlockStream; /** @@ -39,12 +38,12 @@ */ public class OneForOneBlockPusher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockPusher.class); + private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); private final TransportClient client; private final String appId; private final String[] blockIds; private final BlockFetchingListener listener; - private final RpcResponseCallback callback; private final Map buffers; public OneForOneBlockPusher( @@ -57,42 +56,43 @@ public OneForOneBlockPusher( this.appId = appId; this.blockIds = blockIds; this.listener = listener; - this.callback = new BlockPushCallback(); this.buffers = buffers; } private class BlockPushCallback implements RpcResponseCallback { + private int index; + private String blockId; + + BlockPushCallback(int index, String blockId) { + this.index = index; + this.blockId = blockId; + } + @Override public void onSuccess(ByteBuffer response) { - BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); - if (msgObj instanceof PushBlockStream) { - PushBlockStream respMsg = (PushBlockStream) msgObj; - // On receipt of a successful block push - listener.onBlockFetchSuccess(respMsg.blockId, - new NioManagedBuffer(ByteBuffer.allocate(0))); - } else { - throw new UnsupportedOperationException("Unexpected response: " + msgObj); - } + // On receipt of a successful block push + listener.onBlockFetchSuccess(blockId, new NioManagedBuffer(ByteBuffer.allocate(0))); } @Override public void onFailure(Throwable e) { - // Decode PushBlockStream header information from the exception message - BlockPushException decodedExp = BlockPushException.decodeException(e.getMessage()); - PushBlockStream respMsg = decodedExp.getHeader(); - if (respMsg != null) { - // If a PushBlockStream header can be decoded from the exception message, this exception - // is generated by the server, and thus is not retriable. Instead of failing all remaining - // requests, we only fail the one that's actually failed. - String[] targetBlockId = Arrays.copyOfRange(blockIds, respMsg.index, respMsg.index + 1); - failRemainingBlocks(targetBlockId, decodedExp); + // Since block push is best effort, i.e., if we encountered a block push failure that's not + // retriable or exceeding the max retires, we should not fail all remaining block pushes. + // The best effort nature makes block push tolerable of a partial completion. Thus, we only + // fail the block that's actually failed. Not that, on the RetryingBlockFetcher side, once + // retry is initiated, it would still invalidate the previous active retry listener, and + // retry all outstanding blocks. We are preventing forwarding unnecessary block push failures + // to the parent listener of the retry listener. The only exceptions would be if the block + // push failure is due to block arriving on the server side after merge finalization, or the + // client fails to establish connection to the server side. In both cases, we would fail all + // remaining blocks. + if (PUSH_ERROR_HANDLER.shouldRetryError(e)) { + String[] targetBlockId = Arrays.copyOfRange(blockIds, index, index + 1); + failRemainingBlocks(targetBlockId, e); } else { - // If a PushBlockStream header cannot be decoded from the exception message, this - // exception is not generated by the server when processing a PushBlockStream request. - // In this case, the exception could potentially be handled by retry. Since we do not - // have access to index in this case, we have to retry the entire batch. - failRemainingBlocks(blockIds, e); + String[] targetBlockId = Arrays.copyOfRange(blockIds, index, blockIds.length); + failRemainingBlocks(targetBlockId, e); } } } @@ -118,7 +118,8 @@ public void start() { + "is found for this block.", blockIds[i]); } else { ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); - client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), callback); + client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), + new BlockPushCallback(i, blockIds[i])); } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index bbcfb55853760..50207c1e48812 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -231,6 +231,10 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { logger.error( String.format("Failed to fetch block %s, and will not retry (%s retries)", blockId, retryCount), exception); + } else { + logger.debug( + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); } outstandingBlocksIds.remove(blockId); shouldForwardFailure = true; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java deleted file mode 100644 index 801229a509ea1..0000000000000 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockPushExceptionSuite.java +++ /dev/null @@ -1,41 +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; - -import java.io.IOException; - -import com.google.common.base.Throwables; -import org.junit.Test; - -import static org.junit.Assert.*; - -import org.apache.spark.network.shuffle.protocol.PushBlockStream; -import org.apache.spark.network.util.JavaUtils; - -public class BlockPushExceptionSuite { - - @Test - public void testDecodeExceptionMsg() { - PushBlockStream header = new PushBlockStream("app_0", "block_0", 128); - IOException ioexp = new IOException("Test exception"); - String encodedMsg = JavaUtils.encodeHeaderIntoErrorString(header.toByteBuffer(), ioexp); - BlockPushException exp = BlockPushException.decodeException(encodedMsg); - assertEquals(header, exp.getHeader()); - assertEquals(Throwables.getStackTraceAsString(ioexp), exp.getMessage()); - } -} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java index 29b76d6392e5e..b3308fc3b2328 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -31,21 +31,21 @@ public class ErrorHandlerSuite { @Test public void testPushErrorRetry() { ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); - assertFalse(handler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))); + assertFalse(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); assertFalse(handler.shouldRetryError(new RuntimeException(new ConnectException()))); - assertTrue(handler.shouldRetryError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + assertTrue(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); assertTrue(handler.shouldRetryError(new Throwable())); } @Test public void testPushErrorLogging() { ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); - assertFalse(handler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))); - assertFalse(handler.shouldLogError(new RuntimeException( - new IllegalArgumentException(BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); + assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); assertTrue(handler.shouldLogError(new Throwable())); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java index 7c1e408bc5ebb..822f21d2328c1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -37,7 +37,6 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.PushBlockStream; -import org.apache.spark.network.util.JavaUtils; public class OneForOneBlockPusherSuite { @@ -97,11 +96,11 @@ public void testServerFailures() { } @Test - public void testServerClientFailures() { + public void testHandlingRetriableFailures() { LinkedHashMap blocks = Maps.newLinkedHashMap(); blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); - blocks.put("b2", null); + blocks.put("b1", null); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( @@ -112,15 +111,15 @@ public void testServerClientFailures() { new PushBlockStream("app-id", "b2", 2))); verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); - verify(listener, times(1)).onBlockFetchFailure(eq("b0"), any()); - verify(listener, times(2)).onBlockFetchFailure(eq("b1"), any()); - verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + verify(listener, times(0)).onBlockFetchFailure(eq("b0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); } /** * Begins a push on the given set of blocks by mocking the response from server side. - * If a block is an empty byte, a server side exception will be thrown. - * If a block is null, a client side exception will be thrown. + * If a block is an empty byte, a server side retriable exception will be thrown. + * If a block is null, a non-retriable exception will be thrown. */ private static BlockFetchingListener pushBlocks( LinkedHashMap blocks, @@ -142,10 +141,10 @@ private static BlockFetchingListener pushBlocks( if (block != null && block.nioByteBuffer().capacity() > 0) { callback.onSuccess(header); } else if (block != null) { - callback.onFailure(new RuntimeException(JavaUtils.encodeHeaderIntoErrorString(header, - new RuntimeException("Failed " + entry.getKey())))); + callback.onFailure(new RuntimeException("Failed " + entry.getKey())); } else { - callback.onFailure(new RuntimeException("Quick fail " + entry.getKey())); + callback.onFailure(new RuntimeException("Quick fail " + entry.getKey() + + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); } assertEquals(msgIterator.next(), message); return null; From b45b1903b21cc1d3d0fa8e41397bbb5053c596e9 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Sat, 10 Oct 2020 06:04:24 -0700 Subject: [PATCH 25/29] Additional cleanups. --- .../apache/spark/network/server/TransportRequestHandler.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 2be236ebd6815..f178928006902 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -244,8 +244,6 @@ public String getID() { // stream, it's not worth trying to recover. channel.pipeline().fireExceptionCaught(e); } finally { - // Make sure we always release the original metadata buffer by the time we exit the - // invocation of this method. Otherwise, we see memory issues fairly quickly in benchmarks. req.meta.release(); } } From f11eb9b03faa815a81b0fec225941c17fb1720ac Mon Sep 17 00:00:00 2001 From: Min Shen Date: Sun, 11 Oct 2020 10:07:17 -0700 Subject: [PATCH 26/29] Fix additional review comments. --- .../network/shuffle/BlockStoreClient.java | 5 +-- .../spark/network/shuffle/ErrorHandler.java | 23 +++++-------- .../network/shuffle/ExternalBlockHandler.java | 4 +-- .../shuffle/ExternalBlockStoreClient.java | 2 +- .../network/shuffle/OneForOneBlockPusher.java | 13 +++----- .../network/shuffle/RetryingBlockFetcher.java | 8 ++--- .../protocol/FinalizeShuffleMerge.java | 8 ++--- .../shuffle/protocol/MergeStatuses.java | 14 ++++---- .../shuffle/protocol/PushBlockStream.java | 14 ++++---- .../network/shuffle/ErrorHandlerSuite.java | 8 ++--- .../shuffle/ExternalBlockHandlerSuite.java | 4 +-- .../shuffle/OneForOneBlockPusherSuite.java | 33 ++++++++++--------- 12 files changed, 65 insertions(+), 71 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index 1161a3351b3a9..37befcd4b67fa 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -138,8 +138,9 @@ public void onFailure(Throwable t) { } /** - * Push a sequence of shuffle blocks to a remote node asynchronously to be merged into - * a larger shuffle block. + * Push a sequence of shuffle blocks in a best-effort manner to a remote node asynchronously. + * These shuffle blocks, along with blocks pushed by other clients, will be merged into + * per-shuffle partition merged shuffle files on the destination node. * * @param host the host of the remote node. * @param port the port of the remote node. diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java index 993d01cfc49be..10e0065d27420 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -19,6 +19,8 @@ import java.net.ConnectException; +import com.google.common.base.Throwables; + /** * Plugs into {@link RetryingBlockFetcher} to further control when an exception should be retried * and logged. @@ -51,7 +53,7 @@ class BlockPushErrorHandler implements ErrorHandler { * will not retry pushing the block nor log the exception on the client side. */ public static final String TOO_LATE_MESSAGE_SUFFIX = - "received after merged shuffle is finalized"; + "received after merged shuffle is finalized"; /** * String constant used for generating exception messages indicating the server couldn't @@ -61,8 +63,7 @@ class BlockPushErrorHandler implements ErrorHandler { * this reason, we will not log the exception on the client side. */ public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = - "Couldn't find an opportunity to write block"; - + "Couldn't find an opportunity to write block"; @Override public boolean shouldRetryError(Throwable t) { @@ -71,22 +72,14 @@ public boolean shouldRetryError(Throwable t) { return false; } // If the block is too late, there is no need to retry it - return (t.getMessage() == null || - !t.getMessage().contains(TOO_LATE_MESSAGE_SUFFIX)) && - (t.getCause() == null || t.getCause().getMessage() == null || - !t.getCause().getMessage().contains(TOO_LATE_MESSAGE_SUFFIX)); + return !Throwables.getStackTraceAsString(t).contains(TOO_LATE_MESSAGE_SUFFIX); } @Override public boolean shouldLogError(Throwable t) { - return (t.getMessage() == null || - (!t.getMessage().contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && - !t.getMessage().contains(TOO_LATE_MESSAGE_SUFFIX))) && - (t.getCause() == null || t.getCause().getMessage() == null || - (!t.getCause().getMessage() - .contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && - !t.getCause().getMessage() - .contains(TOO_LATE_MESSAGE_SUFFIX))); + String errorStackTrace = Throwables.getStackTraceAsString(t); + return !errorStackTrace.contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + !errorStackTrace.contains(TOO_LATE_MESSAGE_SUFFIX); } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 15ff9c359c3c5..cfe8f82280cfd 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -76,7 +76,7 @@ public ExternalBlockHandler( File registeredExecutorFile, MergedShuffleFileManager mergeManager) throws IOException { this(new OneForOneStreamManager(), - new ExternalShuffleBlockResolver(conf, registeredExecutorFile), mergeManager); + new ExternalShuffleBlockResolver(conf, registeredExecutorFile), mergeManager); } @VisibleForTesting @@ -203,7 +203,7 @@ protected void handleMessage( callback.onSuccess(statuses.toByteBuffer()); } catch(IOException e) { throw new RuntimeException(String.format("Error while finalizing shuffle merge " - + "for application %s shuffle %d", msg.appId, msg.shuffleId)); + + "for application %s shuffle %d", msg.appId, msg.shuffleId)); } finally { responseDelayContext.stop(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index ac5cdd8f864e0..eb32e510d318f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -135,7 +135,7 @@ public void pushBlocks( for (int i = 0; i < blockIds.length; i++) { buffersWithId.put(blockIds[i], buffers[i]); } - logger.debug("Push shuffle blocks to {}:{} with {} blocks", host, port, blockIds.length); + logger.debug("Push {} shuffle blocks to {}:{}", blockIds.length, host, port); try { RetryingBlockFetcher.BlockFetchStarter blockPushStarter = (blockIds1, listener1) -> { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java index 3fc0eb1f367ec..407b248170a46 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -113,14 +113,11 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { public void start() { logger.debug("Start pushing {} blocks", blockIds.length); for (int i = 0; i < blockIds.length; i++) { - if (!buffers.containsKey(blockIds[i])) { - logger.warn("Ignore block push request for block id {} since no buffer " - + "is found for this block.", blockIds[i]); - } else { - ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); - client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), - new BlockPushCallback(i, blockIds[i])); - } + assert buffers.containsKey(blockIds[i]) : "Could not find the block buffer for block " + + blockIds[i]; + ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); + client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), + new BlockPushCallback(i, blockIds[i])); } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index 50207c1e48812..43bde1610e41e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -229,12 +229,12 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { } else { if (errorHandler.shouldLogError(exception)) { logger.error( - String.format("Failed to fetch block %s, and will not retry (%s retries)", - blockId, retryCount), exception); + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); } else { logger.debug( - String.format("Failed to fetch block %s, and will not retry (%s retries)", - blockId, retryCount), exception); + String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); } outstandingBlocksIds.remove(blockId); shouldForwardFailure = true; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java index 563ec60cede25..136678b464084 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java @@ -51,9 +51,9 @@ public int hashCode() { @Override public String toString() { return Objects.toStringHelper(this) - .add("appId", appId) - .add("shuffleId", shuffleId) - .toString(); + .add("appId", appId) + .add("shuffleId", shuffleId) + .toString(); } @Override @@ -61,7 +61,7 @@ public boolean equals(Object other) { if (other != null && other instanceof FinalizeShuffleMerge) { FinalizeShuffleMerge o = (FinalizeShuffleMerge) other; return Objects.equal(appId, o.appId) - && shuffleId == o.shuffleId; + && shuffleId == o.shuffleId; } return false; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java index 7ba059696d40b..24d2ff098df80 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java @@ -70,7 +70,7 @@ protected Type type() { public int hashCode() { int objectHashCode = Objects.hashCode(shuffleId); return (objectHashCode * 41 + Arrays.hashCode(reduceIds) * 41 - + Arrays.hashCode(bitmaps) * 41 + Arrays.hashCode(sizes)); + + Arrays.hashCode(bitmaps) * 41 + Arrays.hashCode(sizes)); } @Override @@ -86,9 +86,9 @@ public boolean equals(Object other) { if (other != null && other instanceof MergeStatuses) { MergeStatuses o = (MergeStatuses) other; return Objects.equal(shuffleId, o.shuffleId) - && Arrays.equals(bitmaps, o.bitmaps) - && Arrays.equals(reduceIds, o.reduceIds) - && Arrays.equals(sizes, o.sizes); + && Arrays.equals(bitmaps, o.bitmaps) + && Arrays.equals(reduceIds, o.reduceIds) + && Arrays.equals(sizes, o.sizes); } return false; } @@ -96,9 +96,9 @@ public boolean equals(Object other) { @Override public int encodedLength() { return 4 // int - + Encoders.BitmapArrays.encodedLength(bitmaps) - + Encoders.IntArrays.encodedLength(reduceIds) - + Encoders.LongArrays.encodedLength(sizes); + + Encoders.BitmapArrays.encodedLength(bitmaps) + + Encoders.IntArrays.encodedLength(reduceIds) + + Encoders.LongArrays.encodedLength(sizes); } @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index 343e2312b979a..7eab5a644783c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -56,10 +56,10 @@ public int hashCode() { @Override public String toString() { return Objects.toStringHelper(this) - .add("appId", appId) - .add("blockId", blockId) - .add("index", index) - .toString(); + .add("appId", appId) + .add("blockId", blockId) + .add("index", index) + .toString(); } @Override @@ -67,8 +67,8 @@ public boolean equals(Object other) { if (other != null && other instanceof PushBlockStream) { PushBlockStream o = (PushBlockStream) other; return Objects.equal(appId, o.appId) - && Objects.equal(blockId, o.blockId) - && index == o.index; + && Objects.equal(blockId, o.blockId) + && index == o.index; } return false; } @@ -76,7 +76,7 @@ public boolean equals(Object other) { @Override public int encodedLength() { return Encoders.Strings.encodedLength(appId) - + Encoders.Strings.encodedLength(blockId) + 4; + + Encoders.Strings.encodedLength(blockId) + 4; } @Override diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java index b3308fc3b2328..276a739e78a4c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -32,10 +32,10 @@ public class ErrorHandlerSuite { public void testPushErrorRetry() { ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); assertFalse(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); assertFalse(handler.shouldRetryError(new RuntimeException(new ConnectException()))); assertTrue(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); assertTrue(handler.shouldRetryError(new Throwable())); } @@ -43,9 +43,9 @@ public void testPushErrorRetry() { public void testPushErrorLogging() { ErrorHandler.BlockPushErrorHandler handler = new ErrorHandler.BlockPushErrorHandler(); assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); assertTrue(handler.shouldLogError(new Throwable())); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 93593bf3a7ca7..680b8d74a2eea 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -238,7 +238,7 @@ public void testFinalizeShuffleMerge() throws IOException { FinalizeShuffleMerge req = new FinalizeShuffleMerge("app0", 0); RoaringBitmap bitmap = RoaringBitmap.bitmapOf(0, 1, 2); MergeStatuses statuses = new MergeStatuses(0, new RoaringBitmap[]{bitmap}, - new int[]{3}, new long[]{30}); + new int[]{3}, new long[]{30}); when(mergedShuffleManager.finalizeShuffleMerge(req)).thenReturn(statuses); ByteBuffer reqBuf = req.toByteBuffer(); @@ -249,7 +249,7 @@ public void testFinalizeShuffleMerge() throws IOException { verify(callback, never()).onFailure(any()); MergeStatuses mergeStatuses = - (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + (MergeStatuses) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); assertEquals(mergeStatuses, statuses); Timer finalizeShuffleMergeLatencyMillis = (Timer) ((ExternalBlockHandler) handler) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java index 822f21d2328c1..09a8a2b352f80 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -28,6 +28,7 @@ import org.junit.Test; import static org.junit.Assert.*; +import static org.mockito.AdditionalMatchers.*; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.ManagedBuffer; @@ -67,8 +68,8 @@ public void testPushThree() { blocks, blockIds, Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); for (int i = 0; i < 3; i ++) { verify(listener, times(1)).onBlockFetchSuccess(eq("b" + i), any()); @@ -84,11 +85,11 @@ public void testServerFailures() { String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( - blocks, - blockIds, - Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); @@ -104,13 +105,14 @@ public void testHandlingRetriableFailures() { String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( - blocks, - blockIds, - Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); + blocks, + blockIds, + Arrays.asList(new PushBlockStream("app-id", "b0", 0), + new PushBlockStream("app-id", "b1", 1), + new PushBlockStream("app-id", "b2", 2))); verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); + verify(listener, times(0)).onBlockFetchSuccess(not(eq("b0")), any()); verify(listener, times(0)).onBlockFetchFailure(eq("b0"), any()); verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); @@ -128,7 +130,7 @@ private static BlockFetchingListener pushBlocks( TransportClient client = mock(TransportClient.class); BlockFetchingListener listener = mock(BlockFetchingListener.class); OneForOneBlockPusher pusher = - new OneForOneBlockPusher(client, "app-id", blockIds, listener, blocks); + new OneForOneBlockPusher(client, "app-id", blockIds, listener, blocks); Iterator> blockIterator = blocks.entrySet().iterator(); Iterator msgIterator = expectMessages.iterator(); @@ -141,10 +143,11 @@ private static BlockFetchingListener pushBlocks( if (block != null && block.nioByteBuffer().capacity() > 0) { callback.onSuccess(header); } else if (block != null) { - callback.onFailure(new RuntimeException("Failed " + entry.getKey())); + callback.onFailure(new RuntimeException("Failed " + entry.getKey() + + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)); } else { callback.onFailure(new RuntimeException("Quick fail " + entry.getKey() - + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); + + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); } assertEquals(msgIterator.next(), message); return null; From 691635e65d11190bfb8cd4b1808bc7e7b78da94d Mon Sep 17 00:00:00 2001 From: Min Shen Date: Mon, 12 Oct 2020 01:22:22 -0700 Subject: [PATCH 27/29] Fix additional review comments --- .../spark/network/shuffle/ErrorHandler.java | 4 ++-- .../network/shuffle/ExternalBlockHandler.java | 2 +- .../network/shuffle/ExternalBlockStoreClient.java | 15 ++++++++------- .../shuffle/protocol/FinalizeShuffleMerge.java | 3 +-- .../spark/network/shuffle/ErrorHandlerSuite.java | 4 ++-- .../shuffle/OneForOneBlockPusherSuite.java | 2 +- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java index 10e0065d27420..308b0b7a6b33b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java @@ -62,7 +62,7 @@ class BlockPushErrorHandler implements ErrorHandler { * side. When we get a block push failure because of the block couldn't be written due to * this reason, we will not log the exception on the client side. */ - public static final String COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX = + public static final String BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX = "Couldn't find an opportunity to write block"; @Override @@ -78,7 +78,7 @@ public boolean shouldRetryError(Throwable t) { @Override public boolean shouldLogError(Throwable t) { String errorStackTrace = Throwables.getStackTraceAsString(t); - return !errorStackTrace.contains(COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) && + return !errorStackTrace.contains(BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX) && !errorStackTrace.contains(TOO_LATE_MESSAGE_SUFFIX); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index cfe8f82280cfd..64eef0fa9a757 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -203,7 +203,7 @@ protected void handleMessage( callback.onSuccess(statuses.toByteBuffer()); } catch(IOException e) { throw new RuntimeException(String.format("Error while finalizing shuffle merge " - + "for application %s shuffle %d", msg.appId, msg.shuffleId)); + + "for application %s shuffle %d", msg.appId, msg.shuffleId), e); } finally { responseDelayContext.stop(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index eb32e510d318f..eca35ed290467 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -29,15 +29,15 @@ import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.shuffle.protocol.*; -import org.apache.spark.network.TransportContext; -import org.apache.spark.network.buffer.ManagedBuffer; 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.*; import org.apache.spark.network.util.TransportConf; /** @@ -95,12 +95,12 @@ public void fetchBlocks( try { int maxRetries = conf.maxIORetries(); RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = - (blockIds1, listener1) -> { + (inputBlockId, inputListener) -> { // Unless this client is closed. if (clientFactory != null) { TransportClient client = clientFactory.createClient(host, port, maxRetries > 0); new OneForOneBlockFetcher(client, appId, execId, - blockIds1, listener1, conf, downloadFileManager).start(); + inputBlockId, inputListener, conf, downloadFileManager).start(); } else { logger.info("This clientFactory was closed. Skipping further block fetch retries."); } @@ -138,9 +138,10 @@ public void pushBlocks( logger.debug("Push {} shuffle blocks to {}:{}", blockIds.length, host, port); try { RetryingBlockFetcher.BlockFetchStarter blockPushStarter = - (blockIds1, listener1) -> { + (inputBlockId, inputListener) -> { TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockPusher(client, appId, blockIds1, listener1, buffersWithId).start(); + new OneForOneBlockPusher(client, appId, inputBlockId, inputListener, buffersWithId) + .start(); }; int maxRetries = conf.maxIORetries(); if (maxRetries > 0) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java index 136678b464084..9058575df57ef 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java @@ -23,8 +23,7 @@ import org.apache.spark.network.protocol.Encoders; /** - * Request from the DAGScheduler to notify external shuffle service to finalize merge - * for a given shuffle. + * Request to finalize merge for a given shuffle. * Returns {@link MergeStatuses} */ public class FinalizeShuffleMerge extends BlockTransferMessage { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java index 276a739e78a4c..992e7762c5a54 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ErrorHandlerSuite.java @@ -35,7 +35,7 @@ public void testPushErrorRetry() { ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); assertFalse(handler.shouldRetryError(new RuntimeException(new ConnectException()))); assertTrue(handler.shouldRetryError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))); assertTrue(handler.shouldRetryError(new Throwable())); } @@ -45,7 +45,7 @@ public void testPushErrorLogging() { assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))); assertFalse(handler.shouldLogError(new RuntimeException(new IllegalArgumentException( - ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)))); + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))); assertTrue(handler.shouldLogError(new Throwable())); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java index 09a8a2b352f80..ebcdba72aa1a8 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -144,7 +144,7 @@ private static BlockFetchingListener pushBlocks( callback.onSuccess(header); } else if (block != null) { callback.onFailure(new RuntimeException("Failed " + entry.getKey() - + ErrorHandler.BlockPushErrorHandler.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX)); + + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)); } else { callback.onFailure(new RuntimeException("Quick fail " + entry.getKey() + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); From f016b395a321b0f44ce9e07d92bcede77553c060 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Mon, 12 Oct 2020 09:29:13 -0700 Subject: [PATCH 28/29] Fix additional review comments. --- .../apache/spark/network/shuffle/ExternalBlockHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 64eef0fa9a757..321b25305c504 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -445,7 +445,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc @Override public void registerApplication(String appId, String user) { - throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + // No-op. Do nothing. } @Override @@ -466,7 +466,7 @@ public ManagedBuffer getMergedBlockData( @Override public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { - return null; + throw new UnsupportedOperationException("Cannot handle shuffle block merge"); } @Override From 2c95f18d2bdf9ac373b0d5319b686a1d66c1e72b Mon Sep 17 00:00:00 2001 From: Min Shen Date: Tue, 13 Oct 2020 08:54:14 -0700 Subject: [PATCH 29/29] Fix styling issue --- .../org/apache/spark/network/shuffle/protocol/MergeStatuses.java | 1 - 1 file changed, 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java index 24d2ff098df80..f57e8b326e5e2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java @@ -25,7 +25,6 @@ import org.apache.spark.network.protocol.Encoders; - /** * Result returned by an ExternalShuffleService to the DAGScheduler. This represents the result * of all the remote shuffle block merge operations performed by an ExternalShuffleService