Skip to content

Commit fab51d4

Browse files
committed
Address comments, add unit test
1 parent 1a2eb32 commit fab51d4

File tree

4 files changed

+41
-11
lines changed

4 files changed

+41
-11
lines changed

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -247,11 +247,7 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
247247
@Override
248248
public String[] getMergedBlockDirs(String appId) {
249249
AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
250-
String[] activeLocalDirs =
251-
Preconditions.checkNotNull(appShuffleInfo.appPathsInfo.activeLocalDirs,
252-
"application " + appId + " active local dirs list has not been updated " +
253-
"by any executor registration");
254-
return activeLocalDirs;
250+
return appShuffleInfo.appPathsInfo.activeLocalDirs;
255251
}
256252

257253
@Override

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ public boolean equals(Object other) {
7676

7777
@Override
7878
public int encodedLength() {
79-
return Encoders.Strings.encodedLength(appId) + 8;
79+
return Encoders.Strings.encodedLength(appId) + 4 + 4;
8080
}
8181

8282
@Override

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222

2323
import org.apache.commons.lang3.builder.ToStringBuilder;
2424
import org.apache.commons.lang3.builder.ToStringStyle;
25+
2526
import org.apache.spark.network.protocol.Encoders;
2627

2728
// Needed by ScalaDoc. See SPARK-7726

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java

Lines changed: 38 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -916,7 +916,7 @@ public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws
916916
}
917917

918918
@Test(expected = IllegalArgumentException.class)
919-
public void testBlockReceivedAfterNewAttemptRegistered()
919+
public void testPushBlockFromPreviousAttemptIsRejected()
920920
throws IOException, InterruptedException {
921921
Semaphore closed = new Semaphore(0);
922922
pushResolver = new RemoteBlockPushResolver(conf) {
@@ -928,7 +928,7 @@ void closeAndDeletePartitionFilesIfNeeded(
928928
closed.release();
929929
}
930930
};
931-
String testApp = "updateLocalDirsTwiceWithTwoAttempts";
931+
String testApp = "testPushBlockFromPreviousAttemptIsRejected";
932932
Path[] attempt1LocalDirs = createLocalDirs(1);
933933
registerExecutor(testApp,
934934
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
@@ -982,13 +982,46 @@ void closeAndDeletePartitionFilesIfNeeded(
982982
assertEquals(
983983
"The attempt id 1 in this PushBlockStream message does not match " +
984984
"with the current attempt id 2 stored in shuffle service for application " +
985-
"updateLocalDirsTwiceWithTwoAttempts", re.getMessage());
985+
testApp, re.getMessage());
986986
throw re;
987987
}
988988
}
989989

990+
@Test(expected = IllegalArgumentException.class)
991+
public void testFinalizeShuffleMergeFromPreviousAttemptIsAborted()
992+
throws IOException, InterruptedException {
993+
String testApp = "testFinalizeShuffleMergeFromPreviousAttemptIsAborted";
994+
Path[] attempt1LocalDirs = createLocalDirs(1);
995+
registerExecutor(testApp,
996+
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
997+
MERGE_DIRECTORY_META_1);
998+
ByteBuffer[] blocks = new ByteBuffer[]{
999+
ByteBuffer.wrap(new byte[4]),
1000+
ByteBuffer.wrap(new byte[5])
1001+
};
1002+
StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
1003+
new PushBlockStream(testApp, 1, 0, 0, 0, 0));
1004+
for (ByteBuffer block : blocks) {
1005+
stream1.onData(stream1.getID(), block);
1006+
}
1007+
stream1.onComplete(stream1.getID());
1008+
Path[] attempt2LocalDirs = createLocalDirs(2);
1009+
registerExecutor(testApp,
1010+
prepareLocalDirs(attempt2LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_2),
1011+
MERGE_DIRECTORY_META_2);
1012+
try {
1013+
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, ATTEMPT_ID_1, 0));
1014+
} catch (IllegalArgumentException e) {
1015+
assertEquals(e.getMessage(),
1016+
String.format("The attempt id %s in this FinalizeShuffleMerge message does not " +
1017+
"match with the current attempt id %s stored in shuffle service for application %s",
1018+
ATTEMPT_ID_1, ATTEMPT_ID_2, testApp));
1019+
throw e;
1020+
}
1021+
}
1022+
9901023
@Test(expected = ClosedChannelException.class)
991-
public void testPushBlockStreamCallBackWhileNewAttemptRegistered()
1024+
public void testOngoingMergeOfBlockFromPreviousAttemptIsAborted()
9921025
throws IOException, InterruptedException {
9931026
Semaphore closed = new Semaphore(0);
9941027
pushResolver = new RemoteBlockPushResolver(conf) {
@@ -1000,7 +1033,7 @@ void closeAndDeletePartitionFilesIfNeeded(
10001033
closed.release();
10011034
}
10021035
};
1003-
String testApp = "testPushBlockStreamCallBackWhileNewAttemptRegisters";
1036+
String testApp = "testOngoingMergeOfBlockFromPreviousAttemptIsAborted";
10041037
Path[] attempt1LocalDirs = createLocalDirs(1);
10051038
registerExecutor(testApp,
10061039
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),

0 commit comments

Comments
 (0)