[SPARK-35546][SHUFFLE] Enable push-based shuffle when multiple app attempts are enabled and manage concurrent access to the state in a better way

### What changes were proposed in this pull request?
This is one of the patches for SPIP SPARK-30602 which is needed for push-based shuffle.

### Summary of the change:
When Executor registers with Shuffle Service, it will encode the merged shuffle dir created and also the application attemptId into the ShuffleManagerMeta into Json. Then in Shuffle Service, it will decode the Json string and get the correct merged shuffle dir and also the attemptId. If the registration comes from a newer attempt, the merged shuffle information will be updated to store the information from the newer attempt.

This PR also refactored the management of the merged shuffle information to avoid concurrency issues.
### Why are the changes needed?
Refer to the SPIP in SPARK-30602.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Added unit tests.
The reference PR with the consolidated changes covering the complete implementation is also provided in SPARK-30602.
We have already verified the functionality and the improved performance as documented in the SPIP doc.

Closes #33078 from zhouyejoe/SPARK-35546.

Authored-by: Ye Zhou <yezhou@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit c77acf0bbc)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
This commit is contained in:
Ye Zhou 2021-07-20 00:03:30 -05:00 committed by Mridul Muralidharan
parent 9d461501b9
commit 1907f0ac57
17 changed files with 810 additions and 391 deletions

View file

@ -419,4 +419,11 @@ public class TransportConf {
public int ioExceptionsThresholdDuringMerge() { public int ioExceptionsThresholdDuringMerge() {
return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4); return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
} }
/**
* The application attemptID assigned from Hadoop YARN.
*/
public int appAttemptId() {
return conf.getInt("spark.app.attempt.id", -1);
}
} }

View file

@ -141,8 +141,8 @@ public class ExternalBlockStoreClient extends BlockStoreClient {
RetryingBlockFetcher.BlockFetchStarter blockPushStarter = RetryingBlockFetcher.BlockFetchStarter blockPushStarter =
(inputBlockId, inputListener) -> { (inputBlockId, inputListener) -> {
TransportClient client = clientFactory.createClient(host, port); TransportClient client = clientFactory.createClient(host, port);
new OneForOneBlockPusher(client, appId, inputBlockId, inputListener, buffersWithId) new OneForOneBlockPusher(client, appId, conf.appAttemptId(), inputBlockId,
.start(); inputListener, buffersWithId).start();
}; };
int maxRetries = conf.maxIORetries(); int maxRetries = conf.maxIORetries();
if (maxRetries > 0) { if (maxRetries > 0) {
@ -168,7 +168,8 @@ public class ExternalBlockStoreClient extends BlockStoreClient {
checkInit(); checkInit();
try { try {
TransportClient client = clientFactory.createClient(host, port); TransportClient client = clientFactory.createClient(host, port);
ByteBuffer finalizeShuffleMerge = new FinalizeShuffleMerge(appId, shuffleId).toByteBuffer(); ByteBuffer finalizeShuffleMerge =
new FinalizeShuffleMerge(appId, conf.appAttemptId(), shuffleId).toByteBuffer();
client.sendRpc(finalizeShuffleMerge, new RpcResponseCallback() { client.sendRpc(finalizeShuffleMerge, new RpcResponseCallback() {
@Override @Override
public void onSuccess(ByteBuffer response) { public void onSuccess(ByteBuffer response) {

View file

@ -45,6 +45,7 @@ public class OneForOneBlockPusher {
private final TransportClient client; private final TransportClient client;
private final String appId; private final String appId;
private final int appAttemptId;
private final String[] blockIds; private final String[] blockIds;
private final BlockFetchingListener listener; private final BlockFetchingListener listener;
private final Map<String, ManagedBuffer> buffers; private final Map<String, ManagedBuffer> buffers;
@ -52,11 +53,13 @@ public class OneForOneBlockPusher {
public OneForOneBlockPusher( public OneForOneBlockPusher(
TransportClient client, TransportClient client,
String appId, String appId,
int appAttemptId,
String[] blockIds, String[] blockIds,
BlockFetchingListener listener, BlockFetchingListener listener,
Map<String, ManagedBuffer> buffers) { Map<String, ManagedBuffer> buffers) {
this.client = client; this.client = client;
this.appId = appId; this.appId = appId;
this.appAttemptId = appAttemptId;
this.blockIds = blockIds; this.blockIds = blockIds;
this.listener = listener; this.listener = listener;
this.buffers = buffers; this.buffers = buffers;
@ -123,7 +126,8 @@ public class OneForOneBlockPusher {
throw new IllegalArgumentException( throw new IllegalArgumentException(
"Unexpected shuffle push block id format: " + blockIds[i]); "Unexpected shuffle push block id format: " + blockIds[i]);
} }
ByteBuffer header = new PushBlockStream(appId, Integer.parseInt(blockIdParts[1]), ByteBuffer header =
new PushBlockStream(appId, appAttemptId, Integer.parseInt(blockIdParts[1]),
Integer.parseInt(blockIdParts[2]), Integer.parseInt(blockIdParts[3]) , i).toByteBuffer(); Integer.parseInt(blockIdParts[2]), Integer.parseInt(blockIdParts[3]) , i).toByteBuffer();
client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]),
new BlockPushCallback(i, blockIds[i])); new BlockPushCallback(i, blockIds[i]));

View file

@ -28,27 +28,26 @@ import java.nio.file.Path;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicReference;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader; import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache; import com.google.common.cache.LoadingCache;
import com.google.common.cache.Weigher; import com.google.common.cache.Weigher;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.roaringbitmap.RoaringBitmap; import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -73,14 +72,22 @@ import org.apache.spark.network.util.TransportConf;
public class RemoteBlockPushResolver implements MergedShuffleFileManager { public class RemoteBlockPushResolver implements MergedShuffleFileManager {
private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
@VisibleForTesting
static final String MERGE_MANAGER_DIR = "merge_manager";
public static final String MERGED_SHUFFLE_FILE_NAME_PREFIX = "shuffleMerged"; public static final String MERGED_SHUFFLE_FILE_NAME_PREFIX = "shuffleMerged";
public static final String SHUFFLE_META_DELIMITER = ":";
public static final String MERGE_DIR_KEY = "mergeDir";
public static final String ATTEMPT_ID_KEY = "attemptId";
private static final int UNDEFINED_ATTEMPT_ID = -1;
private final ConcurrentMap<String, AppPathsInfo> appsPathInfo; /**
private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions; * A concurrent hashmap where the key is the applicationId, and the value includes
* all the merged shuffle information for this application. AppShuffleInfo stores
* the application attemptId, merged shuffle local directories and the metadata
* for actively being merged shuffle partitions.
*/
private final ConcurrentMap<String, AppShuffleInfo> appsShuffleInfo;
private final Executor directoryCleaner; private final Executor mergedShuffleCleaner;
private final TransportConf conf; private final TransportConf conf;
private final int minChunkSize; private final int minChunkSize;
private final int ioExceptionsThresholdDuringMerge; private final int ioExceptionsThresholdDuringMerge;
@ -92,9 +99,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
@SuppressWarnings("UnstableApiUsage") @SuppressWarnings("UnstableApiUsage")
public RemoteBlockPushResolver(TransportConf conf) { public RemoteBlockPushResolver(TransportConf conf) {
this.conf = conf; this.conf = conf;
this.partitions = Maps.newConcurrentMap(); this.appsShuffleInfo = new ConcurrentHashMap<>();
this.appsPathInfo = Maps.newConcurrentMap(); this.mergedShuffleCleaner = Executors.newSingleThreadExecutor(
this.directoryCleaner = Executors.newSingleThreadExecutor(
// Add `spark` prefix because it will run in NM in Yarn mode. // Add `spark` prefix because it will run in NM in Yarn mode.
NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
@ -112,34 +118,59 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
this.errorHandler = new ErrorHandler.BlockPushErrorHandler(); this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
} }
@VisibleForTesting
protected AppShuffleInfo validateAndGetAppShuffleInfo(String appId) {
// TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
AppShuffleInfo appShuffleInfo = appsShuffleInfo.get(appId);
Preconditions.checkArgument(appShuffleInfo != null,
"application " + appId + " is not registered or NM was restarted.");
return appShuffleInfo;
}
/** /**
* Given the appShuffleId and reduceId that uniquely identifies a given shuffle partition of an * Given the appShuffleInfo, shuffleId and reduceId that uniquely identifies a given shuffle
* application, retrieves the associated metadata. If not present and the corresponding merged * partition of an application, retrieves the associated metadata. If not present and the
* shuffle does not exist, initializes the metadata. * corresponding merged shuffle does not exist, initializes the metadata.
*/ */
private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
AppShuffleId appShuffleId, AppShuffleInfo appShuffleInfo,
int shuffleId,
int reduceId) { int reduceId) {
File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId); File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, reduceId);
if (!partitions.containsKey(appShuffleId) && dataFile.exists()) { ConcurrentMap<Integer, Map<Integer, AppShufflePartitionInfo>> partitions =
// If this partition is already finalized then the partitions map will not contain appShuffleInfo.partitions;
// the appShuffleId but the data file would exist. In that case the block is considered late. Map<Integer, AppShufflePartitionInfo> shufflePartitions =
partitions.compute(shuffleId, (id, map) -> {
if (map == null) {
// If this partition is already finalized then the partitions map will not contain the
// shuffleId but the data file would exist. In that case the block is considered late.
if (dataFile.exists()) {
return null; return null;
} }
Map<Integer, AppShufflePartitionInfo> shufflePartitions = return new ConcurrentHashMap<>();
partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap()); } else {
return map;
}
});
if (shufflePartitions == null) {
return null;
}
return shufflePartitions.computeIfAbsent(reduceId, key -> { return shufflePartitions.computeIfAbsent(reduceId, key -> {
// It only gets here when the key is not present in the map. This could either // 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 // 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 // shuffle partition, or after the merged shuffle file is finalized. We handle these
// two cases accordingly by checking if the file already exists. // two cases accordingly by checking if the file already exists.
File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId); File indexFile =
File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId); appShuffleInfo.getMergedShuffleIndexFile(shuffleId, reduceId);
File metaFile =
appShuffleInfo.getMergedShuffleMetaFile(shuffleId, reduceId);
try { try {
if (dataFile.exists()) { if (dataFile.exists()) {
return null; return null;
} else { } else {
return newAppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile); return newAppShufflePartitionInfo(
appShuffleInfo.appId, shuffleId, reduceId, dataFile, indexFile, metaFile);
} }
} catch (IOException e) { } catch (IOException e) {
logger.error( logger.error(
@ -148,26 +179,28 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
indexFile.getAbsolutePath(), metaFile.getAbsolutePath()); indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
throw new RuntimeException( throw new RuntimeException(
String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s " String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s "
+ "reduceId %s", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e); + "reduceId %s", appShuffleInfo.appId, shuffleId, reduceId), e);
} }
}); });
} }
@VisibleForTesting @VisibleForTesting
AppShufflePartitionInfo newAppShufflePartitionInfo( AppShufflePartitionInfo newAppShufflePartitionInfo(
AppShuffleId appShuffleId, String appId,
int shuffleId,
int reduceId, int reduceId,
File dataFile, File dataFile,
File indexFile, File indexFile,
File metaFile) throws IOException { File metaFile) throws IOException {
return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, return new AppShufflePartitionInfo(appId, shuffleId, reduceId, dataFile,
new MergeShuffleFile(indexFile), new MergeShuffleFile(metaFile)); new MergeShuffleFile(indexFile), new MergeShuffleFile(metaFile));
} }
@Override @Override
public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId); AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId); File indexFile =
appShuffleInfo.getMergedShuffleIndexFile(shuffleId, reduceId);
if (!indexFile.exists()) { if (!indexFile.exists()) {
throw new RuntimeException(String.format( throw new RuntimeException(String.format(
"Merged shuffle index file %s not found", indexFile.getPath())); "Merged shuffle index file %s not found", indexFile.getPath()));
@ -175,7 +208,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
int size = (int) indexFile.length(); int size = (int) indexFile.length();
// First entry is the zero offset // First entry is the zero offset
int numChunks = (size / Long.BYTES) - 1; int numChunks = (size / Long.BYTES) - 1;
File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId); File metaFile = appShuffleInfo.getMergedShuffleMetaFile(shuffleId, reduceId);
if (!metaFile.exists()) { if (!metaFile.exists()) {
throw new RuntimeException(String.format("Merged shuffle meta file %s not found", throw new RuntimeException(String.format("Merged shuffle meta file %s not found",
metaFile.getPath())); metaFile.getPath()));
@ -190,13 +223,14 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
@SuppressWarnings("UnstableApiUsage") @SuppressWarnings("UnstableApiUsage")
@Override @Override
public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) { public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId); AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId); File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, reduceId);
if (!dataFile.exists()) { if (!dataFile.exists()) {
throw new RuntimeException(String.format("Merged shuffle data file %s not found", throw new RuntimeException(String.format("Merged shuffle data file %s not found",
dataFile.getPath())); dataFile.getPath()));
} }
File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId); File indexFile =
appShuffleInfo.getMergedShuffleIndexFile(shuffleId, reduceId);
try { try {
// If we get here, the merged shuffle file should have been properly finalized. Thus we can // 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. // use the file length to determine the size of the merged shuffle block.
@ -210,76 +244,51 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
} }
/**
* The logic here is consistent with
* @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(
* org.apache.spark.storage.BlockId, scala.Option)]]
*/
private File getFile(String appId, String filename) {
// TODO: [SPARK-33236] 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.");
File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
appPathsInfo.subDirsPerLocalDir, filename);
logger.debug("Get merged file {}", targetFile.getAbsolutePath());
return targetFile;
}
private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
return getFile(appShuffleId.appId, fileName);
}
private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
return getFile(appShuffleId.appId, indexName);
}
private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
return getFile(appShuffleId.appId, metaName);
}
@Override @Override
public String[] getMergedBlockDirs(String appId) { public String[] getMergedBlockDirs(String appId) {
AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId), AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
"application " + appId + " is not registered or NM was restarted."); return appShuffleInfo.appPathsInfo.activeLocalDirs;
String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
"application " + appId
+ " active local dirs list has not been updated by any executor registration");
return activeLocalDirs;
} }
@Override @Override
public void applicationRemoved(String appId, boolean cleanupLocalDirs) { public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
// TODO: [SPARK-33236] Change the message when this service is able to handle NM restart AppShuffleInfo appShuffleInfo = appsShuffleInfo.remove(appId);
AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId), if (null != appShuffleInfo) {
"application " + appId + " is not registered or NM was restarted."); mergedShuffleCleaner.execute(
Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator = () -> closeAndDeletePartitionFilesIfNeeded(appShuffleInfo, cleanupLocalDirs));
partitions.entrySet().iterator(); }
while (iterator.hasNext()) { }
Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
AppShuffleId appShuffleId = entry.getKey();
if (appId.equals(appShuffleId.appId)) { /**
iterator.remove(); * Clean up the AppShufflePartitionInfo for a specific AppShuffleInfo.
for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) { * If cleanupLocalDirs is true, the merged shuffle files will also be deleted.
* The cleanup will be executed in a separate thread.
*/
@VisibleForTesting
void closeAndDeletePartitionFilesIfNeeded(
AppShuffleInfo appShuffleInfo,
boolean cleanupLocalDirs) {
for (Map<Integer, AppShufflePartitionInfo> partitionMap : appShuffleInfo.partitions.values()) {
for (AppShufflePartitionInfo partitionInfo : partitionMap.values()) {
synchronized (partitionInfo) {
partitionInfo.closeAllFiles(); partitionInfo.closeAllFiles();
} }
} }
} }
if (cleanupLocalDirs) { if (cleanupLocalDirs) {
Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) deleteExecutorDirs(appShuffleInfo);
.map(dir -> Paths.get(dir)).toArray(Path[]::new);
directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
} }
} }
/** /**
* Serially delete local dirs, executed in a separate thread. * Serially delete local dirs.
*/ */
@VisibleForTesting @VisibleForTesting
void deleteExecutorDirs(Path[] dirs) { void deleteExecutorDirs(AppShuffleInfo appShuffleInfo) {
Path[] dirs = Arrays.stream(appShuffleInfo.appPathsInfo.activeLocalDirs)
.map(dir -> Paths.get(dir)).toArray(Path[]::new);
for (Path localDir : dirs) { for (Path localDir : dirs) {
try { try {
if (Files.exists(localDir)) { if (Files.exists(localDir)) {
@ -294,10 +303,22 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
@Override @Override
public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(msg.appId);
final String streamId = String.format("%s_%d_%d_%d",
OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex,
msg.reduceId);
if (appShuffleInfo.attemptId != msg.appAttemptId) {
// If this Block belongs to a former application attempt, it is considered late,
// as only the blocks from the current application attempt will be merged
// TODO: [SPARK-35548] Client should be updated to handle this error.
throw new IllegalArgumentException(
String.format("The attempt id %s in this PushBlockStream message does not match "
+ "with the current attempt id %s stored in shuffle service for application %s",
msg.appAttemptId, appShuffleInfo.attemptId, msg.appId));
}
// Retrieve merged shuffle file metadata // Retrieve merged shuffle file metadata
AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
AppShufflePartitionInfo partitionInfoBeforeCheck = AppShufflePartitionInfo partitionInfoBeforeCheck =
getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId); getOrCreateAppShufflePartitionInfo(appShuffleInfo, msg.shuffleId, msg.reduceId);
// Here partitionInfo will be null in 2 cases: // 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 // 1) The request is received for a block that has already been merged, this is possible due
// to the retry logic. // to the retry logic.
@ -338,11 +359,9 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
&& partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
: partitionInfoBeforeCheck; : partitionInfoBeforeCheck;
final String streamId = String.format("%s_%d_%d_%d",
OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
msg.reduceId);
if (partitionInfo != null) { if (partitionInfo != null) {
return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex); return new PushBlockStreamCallback(
this, appShuffleInfo, streamId, partitionInfo, msg.mapIndex);
} else { } else {
// For a duplicate block or a block which is late, respond back with a callback that handles // For a duplicate block or a block which is late, respond back with a callback that handles
// them differently. // them differently.
@ -377,24 +396,31 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
} }
@SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
@Override @Override
public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId); logger.info("Finalizing shuffle {} from Application {}_{}.",
AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId); msg.shuffleId, msg.appId, msg.appAttemptId);
Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId); AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(msg.appId);
if (appShuffleInfo.attemptId != msg.appAttemptId) {
// If this Block belongs to a former application attempt, it is considered late,
// as only the blocks from the current application attempt will be merged
// TODO: [SPARK-35548] Client should be updated to handle this error.
throw new IllegalArgumentException(
String.format("The attempt id %s in this FinalizeShuffleMerge message does not match "
+ "with the current attempt id %s stored in shuffle service for application %s",
msg.appAttemptId, appShuffleInfo.attemptId, msg.appId));
}
Map<Integer, AppShufflePartitionInfo> shufflePartitions =
appShuffleInfo.partitions.remove(msg.shuffleId);
MergeStatuses mergeStatuses; MergeStatuses mergeStatuses;
if (shufflePartitions == null || shufflePartitions.isEmpty()) { if (shufflePartitions == null || shufflePartitions.isEmpty()) {
mergeStatuses = mergeStatuses =
new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]); new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
} else { } else {
Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values(); List<RoaringBitmap> bitmaps = new ArrayList<>(shufflePartitions.size());
List<RoaringBitmap> bitmaps = new ArrayList<>(partitionsToFinalize.size()); List<Integer> reduceIds = new ArrayList<>(shufflePartitions.size());
List<Integer> reduceIds = new ArrayList<>(partitionsToFinalize.size()); List<Long> sizes = new ArrayList<>(shufflePartitions.size());
List<Long> sizes = new ArrayList<>(partitionsToFinalize.size()); for (AppShufflePartitionInfo partition: shufflePartitions.values()) {
Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
while (partitionsIter.hasNext()) {
AppShufflePartitionInfo partition = partitionsIter.next();
synchronized (partition) { synchronized (partition) {
try { try {
// This can throw IOException which will marks this shuffle partition as not merged. // This can throw IOException which will marks this shuffle partition as not merged.
@ -403,13 +429,10 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
reduceIds.add(partition.reduceId); reduceIds.add(partition.reduceId);
sizes.add(partition.getLastChunkOffset()); sizes.add(partition.getLastChunkOffset());
} catch (IOException ioe) { } catch (IOException ioe) {
logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId, logger.warn("Exception while finalizing shuffle partition {}_{} {} {}", msg.appId,
msg.shuffleId, partition.reduceId, ioe); msg.appAttemptId, msg.shuffleId, partition.reduceId, ioe);
} finally { } finally {
partition.closeAllFiles(); partition.closeAllFiles();
// The partition should be removed after the files are written so that any new stream
// for the same reduce partition will see that the data file exists.
partitionsIter.remove();
} }
} }
} }
@ -417,8 +440,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds), bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
Longs.toArray(sizes)); Longs.toArray(sizes));
} }
partitions.remove(appShuffleId); logger.info("Finalized shuffle {} from Application {}_{}.",
logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId); msg.shuffleId, msg.appId, msg.appAttemptId);
return mergeStatuses; return mergeStatuses;
} }
@ -426,15 +449,68 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} " logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+ "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs), + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
executorInfo.subDirsPerLocalDir); executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
} }
appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs, String shuffleManagerMeta = executorInfo.shuffleManager;
executorInfo.subDirsPerLocalDir)); if (shuffleManagerMeta.contains(SHUFFLE_META_DELIMITER)) {
String mergeDirInfo =
shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(SHUFFLE_META_DELIMITER) + 1);
try {
ObjectMapper mapper = new ObjectMapper();
TypeReference<Map<String, String>> typeRef
= new TypeReference<Map<String, String>>(){};
Map<String, String> metaMap = mapper.readValue(mergeDirInfo, typeRef);
String mergeDir = metaMap.get(MERGE_DIR_KEY);
int attemptId = Integer.valueOf(
metaMap.getOrDefault(ATTEMPT_ID_KEY, String.valueOf(UNDEFINED_ATTEMPT_ID)));
if (mergeDir == null) {
throw new IllegalArgumentException(
String.format("Failed to get the merge directory information from the " +
"shuffleManagerMeta %s in executor registration message", shuffleManagerMeta));
}
if (attemptId == UNDEFINED_ATTEMPT_ID) {
// When attemptId is -1, there is no attemptId stored in the ExecutorShuffleInfo.
// Only the first ExecutorRegister message can register the merge dirs
appsShuffleInfo.computeIfAbsent(appId, id ->
new AppShuffleInfo(
appId, UNDEFINED_ATTEMPT_ID,
new AppPathsInfo(appId, executorInfo.localDirs,
mergeDir, executorInfo.subDirsPerLocalDir)
));
} else {
// If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
// The first ExecutorRegister message from the same application attempt wil register
// the merge dirs in External Shuffle Service. Any later ExecutorRegister message
// from the same application attempt will not override the merge dirs. But it can
// be overridden by ExecutorRegister message from newer application attempt,
// and former attempts' shuffle partitions information will also be cleaned up.
AtomicReference<AppShuffleInfo> originalAppShuffleInfo = new AtomicReference<>();
appsShuffleInfo.compute(appId, (id, appShuffleInfo) -> {
if (appShuffleInfo == null || attemptId > appShuffleInfo.attemptId) {
originalAppShuffleInfo.set(appShuffleInfo);
appShuffleInfo =
new AppShuffleInfo(
appId, attemptId,
new AppPathsInfo(appId, executorInfo.localDirs,
mergeDir, executorInfo.subDirsPerLocalDir));
}
return appShuffleInfo;
});
if (originalAppShuffleInfo.get() != null) {
AppShuffleInfo appShuffleInfo = originalAppShuffleInfo.get();
logger.warn("Cleanup shuffle info and merged shuffle files for {}_{} as new " +
"application attempt registered", appId, appShuffleInfo.attemptId);
mergedShuffleCleaner.execute(
() -> closeAndDeletePartitionFilesIfNeeded(appShuffleInfo, true));
}
}
} catch (JsonProcessingException e) {
logger.warn("Failed to get the merge directory information from ExecutorShuffleInfo: ", e);
}
} else {
logger.warn("ExecutorShuffleInfo does not have the expected merge directory information");
} }
private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
return String.format("%s_%s_%d_%d", MERGED_SHUFFLE_FILE_NAME_PREFIX, appShuffleId.appId,
appShuffleId.shuffleId, reduceId);
} }
/** /**
@ -443,6 +519,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
static class PushBlockStreamCallback implements StreamCallbackWithID { static class PushBlockStreamCallback implements StreamCallbackWithID {
private final RemoteBlockPushResolver mergeManager; private final RemoteBlockPushResolver mergeManager;
private final AppShuffleInfo appShuffleInfo;
private final String streamId; private final String streamId;
private final int mapIndex; private final int mapIndex;
private final AppShufflePartitionInfo partitionInfo; private final AppShufflePartitionInfo partitionInfo;
@ -457,12 +534,17 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
private PushBlockStreamCallback( private PushBlockStreamCallback(
RemoteBlockPushResolver mergeManager, RemoteBlockPushResolver mergeManager,
AppShuffleInfo appShuffleInfo,
String streamId, String streamId,
AppShufflePartitionInfo partitionInfo, AppShufflePartitionInfo partitionInfo,
int mapIndex) { int mapIndex) {
this.mergeManager = Preconditions.checkNotNull(mergeManager); Preconditions.checkArgument(mergeManager != null);
this.mergeManager = mergeManager;
Preconditions.checkArgument(appShuffleInfo != null);
this.appShuffleInfo = appShuffleInfo;
this.streamId = streamId; this.streamId = streamId;
this.partitionInfo = Preconditions.checkNotNull(partitionInfo); Preconditions.checkArgument(partitionInfo != null);
this.partitionInfo = partitionInfo;
this.mapIndex = mapIndex; this.mapIndex = mapIndex;
abortIfNecessary(); abortIfNecessary();
} }
@ -482,7 +564,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
while (buf.hasRemaining()) { while (buf.hasRemaining()) {
long updatedPos = partitionInfo.getDataFilePos() + length; long updatedPos = partitionInfo.getDataFilePos() + length;
logger.debug("{} shuffleId {} reduceId {} current pos {} updated pos {}", logger.debug("{} shuffleId {} reduceId {} current pos {} updated pos {}",
partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.appId, partitionInfo.shuffleId,
partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos); partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos);
length += partitionInfo.dataChannel.write(buf, updatedPos); length += partitionInfo.dataChannel.write(buf, updatedPos);
} }
@ -567,7 +649,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
// memory, while still providing the necessary guarantee. // memory, while still providing the necessary guarantee.
synchronized (partitionInfo) { synchronized (partitionInfo) {
Map<Integer, AppShufflePartitionInfo> shufflePartitions = Map<Integer, AppShufflePartitionInfo> shufflePartitions =
mergeManager.partitions.get(partitionInfo.appShuffleId); appShuffleInfo.partitions.get(partitionInfo.shuffleId);
// If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
// then it means that the shuffle merge has already been finalized. We should thus ignore // then it means that 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 // the data and just drain the remaining bytes of this message. This check should be
@ -587,7 +669,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
abortIfNecessary(); abortIfNecessary();
logger.trace("{} shuffleId {} reduceId {} onData writable", logger.trace("{} shuffleId {} reduceId {} onData writable",
partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.appId, partitionInfo.shuffleId,
partitionInfo.reduceId); partitionInfo.reduceId);
if (partitionInfo.getCurrentMapIndex() < 0) { if (partitionInfo.getCurrentMapIndex() < 0) {
partitionInfo.setCurrentMapIndex(mapIndex); partitionInfo.setCurrentMapIndex(mapIndex);
@ -609,7 +691,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
} else { } else {
logger.trace("{} shuffleId {} reduceId {} onData deferred", logger.trace("{} shuffleId {} reduceId {} onData deferred",
partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.appId, partitionInfo.shuffleId,
partitionInfo.reduceId); partitionInfo.reduceId);
// If we cannot write to disk, we buffer the current block chunk in memory so it could // 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 // potentially be written to disk later. We take our best effort without guarantee
@ -644,10 +726,10 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
public void onComplete(String streamId) throws IOException { public void onComplete(String streamId) throws IOException {
synchronized (partitionInfo) { synchronized (partitionInfo) {
logger.trace("{} shuffleId {} reduceId {} onComplete invoked", logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.appId, partitionInfo.shuffleId,
partitionInfo.reduceId); partitionInfo.reduceId);
Map<Integer, AppShufflePartitionInfo> shufflePartitions = Map<Integer, AppShufflePartitionInfo> shufflePartitions =
mergeManager.partitions.get(partitionInfo.appShuffleId); appShuffleInfo.partitions.get(partitionInfo.shuffleId);
// When this request initially got to the server, the shuffle merge finalize request // 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 // 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. // however is already finalized. We should thus respond RpcFailure to the client.
@ -724,10 +806,10 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
if (isWriting) { if (isWriting) {
synchronized (partitionInfo) { synchronized (partitionInfo) {
Map<Integer, AppShufflePartitionInfo> shufflePartitions = Map<Integer, AppShufflePartitionInfo> shufflePartitions =
mergeManager.partitions.get(partitionInfo.appShuffleId); appShuffleInfo.partitions.get(partitionInfo.shuffleId);
if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) { if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
logger.debug("{} shuffleId {} reduceId {} encountered failure", logger.debug("{} shuffleId {} reduceId {} encountered failure",
partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.appId, partitionInfo.shuffleId,
partitionInfo.reduceId); partitionInfo.reduceId);
partitionInfo.setCurrentMapIndex(-1); partitionInfo.setCurrentMapIndex(-1);
} }
@ -742,63 +824,25 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
} }
/**
* ID that uniquely identifies a shuffle for an application. This is used as a key in
* {@link #partitions}.
*/
public static class AppShuffleId {
public final String appId;
public final int shuffleId;
AppShuffleId(String appId, int shuffleId) {
this.appId = appId;
this.shuffleId = shuffleId;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
AppShuffleId that = (AppShuffleId) o;
return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
}
@Override
public int hashCode() {
return Objects.hashCode(appId, shuffleId);
}
@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId)
.append("shuffleId", shuffleId)
.toString();
}
}
/** Metadata tracked for an actively merged shuffle partition */ /** Metadata tracked for an actively merged shuffle partition */
public static class AppShufflePartitionInfo { public static class AppShufflePartitionInfo {
private final AppShuffleId appShuffleId; private final String appId;
private final int shuffleId;
private final int reduceId; private final int reduceId;
// The merged shuffle data file channel // The merged shuffle data file channel
public FileChannel dataChannel; public final FileChannel dataChannel;
// The index file for a particular merged shuffle contains the chunk offsets.
private final MergeShuffleFile indexFile;
// The meta file for a particular merged shuffle contains all the map indices that belong to
// every chunk. The entry per chunk is a serialized bitmap.
private final MergeShuffleFile metaFile;
// Location offset of the last successfully merged block for this shuffle partition // Location offset of the last successfully merged block for this shuffle partition
private long dataFilePos; private long dataFilePos;
// Track the map index whose block is being merged for this shuffle partition // Track the map index whose block is being merged for this shuffle partition
private int currentMapIndex; private int currentMapIndex;
// Bitmap tracking which mapper's blocks have been merged for this shuffle partition // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
private RoaringBitmap mapTracker; private RoaringBitmap mapTracker;
// The index file for a particular merged shuffle contains the chunk offsets.
private MergeShuffleFile indexFile;
// The meta file for a particular merged shuffle contains all the map indices that belong to
// every chunk. The entry per chunk is a serialized bitmap.
private MergeShuffleFile metaFile;
// The offset for the last chunk tracked in the index file for this shuffle partition // The offset for the last chunk tracked in the index file for this shuffle partition
private long lastChunkOffset; private long lastChunkOffset;
private int lastMergedMapIndex = -1; private int lastMergedMapIndex = -1;
@ -808,12 +852,15 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
private boolean indexMetaUpdateFailed; private boolean indexMetaUpdateFailed;
AppShufflePartitionInfo( AppShufflePartitionInfo(
AppShuffleId appShuffleId, String appId,
int shuffleId,
int reduceId, int reduceId,
File dataFile, File dataFile,
MergeShuffleFile indexFile, MergeShuffleFile indexFile,
MergeShuffleFile metaFile) throws IOException { MergeShuffleFile metaFile) throws IOException {
this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id"); Preconditions.checkArgument(appId != null, "app id is null");
this.appId = appId;
this.shuffleId = shuffleId;
this.reduceId = reduceId; this.reduceId = reduceId;
this.dataChannel = new FileOutputStream(dataFile).getChannel(); this.dataChannel = new FileOutputStream(dataFile).getChannel();
this.indexFile = indexFile; this.indexFile = indexFile;
@ -831,8 +878,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
public void setDataFilePos(long dataFilePos) { public void setDataFilePos(long dataFilePos) {
logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId, logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appId,
appShuffleId.shuffleId, reduceId, this.dataFilePos, dataFilePos); shuffleId, reduceId, this.dataFilePos, dataFilePos);
this.dataFilePos = dataFilePos; this.dataFilePos = dataFilePos;
} }
@ -842,7 +889,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
void setCurrentMapIndex(int mapIndex) { void setCurrentMapIndex(int mapIndex) {
logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}", logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex); appId, shuffleId, reduceId, currentMapIndex, mapIndex);
this.currentMapIndex = mapIndex; this.currentMapIndex = mapIndex;
} }
@ -851,8 +898,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
void blockMerged(int mapIndex) { void blockMerged(int mapIndex) {
logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId, logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appId,
appShuffleId.shuffleId, reduceId, mapIndex); shuffleId, reduceId, mapIndex);
mapTracker.add(mapIndex); mapTracker.add(mapIndex);
chunkTracker.add(mapIndex); chunkTracker.add(mapIndex);
lastMergedMapIndex = mapIndex; lastMergedMapIndex = mapIndex;
@ -871,7 +918,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException { void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
try { try {
logger.trace("{} shuffleId {} reduceId {} index current {} updated {}", logger.trace("{} shuffleId {} reduceId {} index current {} updated {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset, chunkOffset); appId, shuffleId, reduceId, this.lastChunkOffset, chunkOffset);
if (indexMetaUpdateFailed) { if (indexMetaUpdateFailed) {
indexFile.getChannel().position(indexFile.getPos()); indexFile.getChannel().position(indexFile.getPos());
} }
@ -885,8 +932,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
this.lastChunkOffset = chunkOffset; this.lastChunkOffset = chunkOffset;
indexMetaUpdateFailed = false; indexMetaUpdateFailed = false;
} catch (IOException ioe) { } catch (IOException ioe) {
logger.warn("{} shuffleId {} reduceId {} update to index/meta failed", appShuffleId.appId, logger.warn("{} shuffleId {} reduceId {} update to index/meta failed", appId,
appShuffleId.shuffleId, reduceId); shuffleId, reduceId);
indexMetaUpdateFailed = true; indexMetaUpdateFailed = true;
// Any exception here is propagated to the caller and the caller can decide whether to // Any exception here is propagated to the caller and the caller can decide whether to
// abort or not. // abort or not.
@ -900,7 +947,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
chunkTracker.add(mapIndex); chunkTracker.add(mapIndex);
logger.trace("{} shuffleId {} reduceId {} mapIndex {} write chunk to meta file", logger.trace("{} shuffleId {} reduceId {} mapIndex {} write chunk to meta file",
appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex); appId, shuffleId, reduceId, mapIndex);
if (indexMetaUpdateFailed) { if (indexMetaUpdateFailed) {
metaFile.getChannel().position(metaFile.getPos()); metaFile.getChannel().position(metaFile.getPos());
} }
@ -934,35 +981,25 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
void closeAllFiles() { void closeAllFiles() {
if (dataChannel != null) {
try { try {
if (dataChannel.isOpen()) {
dataChannel.close(); dataChannel.close();
}
} catch (IOException ioe) { } catch (IOException ioe) {
logger.warn("Error closing data channel for {} shuffleId {} reduceId {}", logger.warn("Error closing data channel for {} shuffleId {} reduceId {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId); appId, shuffleId, reduceId);
} finally {
dataChannel = null;
} }
}
if (metaFile != null) {
try { try {
metaFile.close(); metaFile.close();
} catch (IOException ioe) { } catch (IOException ioe) {
logger.warn("Error closing meta file for {} shuffleId {} reduceId {}", logger.warn("Error closing meta file for {} shuffleId {} reduceId {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId); appId, shuffleId, reduceId);
} finally {
metaFile = null;
} }
}
if (indexFile != null) {
try { try {
indexFile.close(); indexFile.close();
} catch (IOException ioe) { } catch (IOException ioe) {
logger.warn("Error closing index file for {} shuffleId {} reduceId {}", logger.warn("Error closing index file for {} shuffleId {} reduceId {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId); appId, shuffleId, reduceId);
} finally {
indexFile = null;
}
} }
} }
@ -1003,14 +1040,16 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
private AppPathsInfo( private AppPathsInfo(
String appId, String appId,
String[] localDirs, String[] localDirs,
String mergeDirectory,
int subDirsPerLocalDir) { int subDirsPerLocalDir) {
activeLocalDirs = Arrays.stream(localDirs) activeLocalDirs = Arrays.stream(localDirs)
.map(localDir -> .map(localDir ->
// Merge directory is created at the same level as block-manager directory. The list of // Merge directory is created at the same level as block-manager directory. The list of
// local directories that we get from executorShuffleInfo are paths of each // local directories that we get from ExecutorShuffleInfo are paths of each
// block-manager directory. To find out the merge directory location, we first find the // block-manager directory. The mergeDirectory is the merge directory name that we get
// parent dir and then append the "merger_manager" directory to it. // from ExecutorShuffleInfo. To find out the merge directory location, we first find the
Paths.get(localDir).getParent().resolve(MERGE_MANAGER_DIR).toFile().getPath()) // parent dir of the block-manager directory and then append merge directory name to it.
Paths.get(localDir).getParent().resolve(mergeDirectory).toFile().getPath())
.toArray(String[]::new); .toArray(String[]::new);
this.subDirsPerLocalDir = subDirsPerLocalDir; this.subDirsPerLocalDir = subDirsPerLocalDir;
if (logger.isInfoEnabled()) { if (logger.isInfoEnabled()) {
@ -1020,10 +1059,76 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
} }
/** Merged Shuffle related information tracked for a specific application attempt */
public static class AppShuffleInfo {
private final String appId;
private final int attemptId;
private final AppPathsInfo appPathsInfo;
private final ConcurrentMap<Integer, Map<Integer, AppShufflePartitionInfo>> partitions;
AppShuffleInfo(
String appId,
int attemptId,
AppPathsInfo appPathsInfo) {
this.appId = appId;
this.attemptId = attemptId;
this.appPathsInfo = appPathsInfo;
partitions = new ConcurrentHashMap<>();
}
@VisibleForTesting
public ConcurrentMap<Integer, Map<Integer, AppShufflePartitionInfo>> getPartitions() {
return partitions;
}
/**
* The logic here is consistent with
* @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(
* org.apache.spark.storage.BlockId, scala.Option)]]
*/
private File getFile(String filename) {
// TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
appPathsInfo.subDirsPerLocalDir, filename);
logger.debug("Get merged file {}", targetFile.getAbsolutePath());
return targetFile;
}
private String generateFileName(
String appId,
int shuffleId,
int reduceId) {
return String.format(
"%s_%s_%d_%d", MERGED_SHUFFLE_FILE_NAME_PREFIX, appId, shuffleId, reduceId);
}
public File getMergedShuffleDataFile(
int shuffleId,
int reduceId) {
String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
return getFile(fileName);
}
public File getMergedShuffleIndexFile(
int shuffleId,
int reduceId) {
String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
return getFile(indexName);
}
public File getMergedShuffleMetaFile(
int shuffleId,
int reduceId) {
String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
return getFile(metaName);
}
}
@VisibleForTesting @VisibleForTesting
static class MergeShuffleFile { static class MergeShuffleFile {
private FileChannel channel; private final FileChannel channel;
private DataOutputStream dos; private final DataOutputStream dos;
private long pos; private long pos;
@VisibleForTesting @VisibleForTesting
@ -1044,11 +1149,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager {
} }
void close() throws IOException { void close() throws IOException {
try { if (channel.isOpen()) {
dos.close(); dos.close();
} finally {
dos = null;
channel = null;
} }
} }

View file

@ -35,7 +35,12 @@ public class ExecutorShuffleInfo implements Encodable {
public final String[] localDirs; public final String[] localDirs;
/** Number of subdirectories created within each localDir. */ /** Number of subdirectories created within each localDir. */
public final int subDirsPerLocalDir; public final int subDirsPerLocalDir;
/** Shuffle manager (SortShuffleManager) that the executor is using. */ /**
* Shuffle manager (SortShuffleManager) that the executor is using.
* If this string contains semicolon, it will also include the meta information
* for push based shuffle in JSON format. Example of the string with semicolon would be:
* SortShuffleManager:{"mergeDir": "mergeDirectory_1", "attemptId": 1}
*/
public final String shuffleManager; public final String shuffleManager;
@JsonCreator @JsonCreator

View file

@ -32,12 +32,15 @@ import org.apache.spark.network.protocol.Encoders;
*/ */
public class FinalizeShuffleMerge extends BlockTransferMessage { public class FinalizeShuffleMerge extends BlockTransferMessage {
public final String appId; public final String appId;
public final int appAttemptId;
public final int shuffleId; public final int shuffleId;
public FinalizeShuffleMerge( public FinalizeShuffleMerge(
String appId, String appId,
int appAttemptId,
int shuffleId) { int shuffleId) {
this.appId = appId; this.appId = appId;
this.appAttemptId = appAttemptId;
this.shuffleId = shuffleId; this.shuffleId = shuffleId;
} }
@ -48,13 +51,14 @@ public class FinalizeShuffleMerge extends BlockTransferMessage {
@Override @Override
public int hashCode() { public int hashCode() {
return Objects.hashCode(appId, shuffleId); return Objects.hashCode(appId, appAttemptId, shuffleId);
} }
@Override @Override
public String toString() { public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId) .append("appId", appId)
.append("attemptId", appAttemptId)
.append("shuffleId", shuffleId) .append("shuffleId", shuffleId)
.toString(); .toString();
} }
@ -64,6 +68,7 @@ public class FinalizeShuffleMerge extends BlockTransferMessage {
if (other != null && other instanceof FinalizeShuffleMerge) { if (other != null && other instanceof FinalizeShuffleMerge) {
FinalizeShuffleMerge o = (FinalizeShuffleMerge) other; FinalizeShuffleMerge o = (FinalizeShuffleMerge) other;
return Objects.equal(appId, o.appId) return Objects.equal(appId, o.appId)
&& appAttemptId == appAttemptId
&& shuffleId == o.shuffleId; && shuffleId == o.shuffleId;
} }
return false; return false;
@ -71,18 +76,20 @@ public class FinalizeShuffleMerge extends BlockTransferMessage {
@Override @Override
public int encodedLength() { public int encodedLength() {
return Encoders.Strings.encodedLength(appId) + 4; return Encoders.Strings.encodedLength(appId) + 4 + 4;
} }
@Override @Override
public void encode(ByteBuf buf) { public void encode(ByteBuf buf) {
Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, appId);
buf.writeInt(appAttemptId);
buf.writeInt(shuffleId); buf.writeInt(shuffleId);
} }
public static FinalizeShuffleMerge decode(ByteBuf buf) { public static FinalizeShuffleMerge decode(ByteBuf buf) {
String appId = Encoders.Strings.decode(buf); String appId = Encoders.Strings.decode(buf);
int attemptId = buf.readInt();
int shuffleId = buf.readInt(); int shuffleId = buf.readInt();
return new FinalizeShuffleMerge(appId, shuffleId); return new FinalizeShuffleMerge(appId, attemptId, shuffleId);
} }
} }

View file

@ -19,6 +19,7 @@ package org.apache.spark.network.shuffle.protocol;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.commons.lang3.builder.ToStringStyle;
@ -34,6 +35,7 @@ import org.apache.spark.network.protocol.Encoders;
*/ */
public class PushBlockStream extends BlockTransferMessage { public class PushBlockStream extends BlockTransferMessage {
public final String appId; public final String appId;
public final int appAttemptId;
public final int shuffleId; public final int shuffleId;
public final int mapIndex; public final int mapIndex;
public final int reduceId; public final int reduceId;
@ -41,8 +43,15 @@ public class PushBlockStream extends BlockTransferMessage {
// blocks to be pushed. // blocks to be pushed.
public final int index; public final int index;
public PushBlockStream(String appId, int shuffleId, int mapIndex, int reduceId, int index) { public PushBlockStream(
String appId,
int appAttemptId,
int shuffleId,
int mapIndex,
int reduceId,
int index) {
this.appId = appId; this.appId = appId;
this.appAttemptId = appAttemptId;
this.shuffleId = shuffleId; this.shuffleId = shuffleId;
this.mapIndex = mapIndex; this.mapIndex = mapIndex;
this.reduceId = reduceId; this.reduceId = reduceId;
@ -56,13 +65,14 @@ public class PushBlockStream extends BlockTransferMessage {
@Override @Override
public int hashCode() { public int hashCode() {
return Objects.hashCode(appId, shuffleId, mapIndex , reduceId, index); return Objects.hashCode(appId, appAttemptId, shuffleId, mapIndex , reduceId, index);
} }
@Override @Override
public String toString() { public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId) .append("appId", appId)
.append("attemptId", appAttemptId)
.append("shuffleId", shuffleId) .append("shuffleId", shuffleId)
.append("mapIndex", mapIndex) .append("mapIndex", mapIndex)
.append("reduceId", reduceId) .append("reduceId", reduceId)
@ -75,6 +85,7 @@ public class PushBlockStream extends BlockTransferMessage {
if (other != null && other instanceof PushBlockStream) { if (other != null && other instanceof PushBlockStream) {
PushBlockStream o = (PushBlockStream) other; PushBlockStream o = (PushBlockStream) other;
return Objects.equal(appId, o.appId) return Objects.equal(appId, o.appId)
&& appAttemptId == o.appAttemptId
&& shuffleId == o.shuffleId && shuffleId == o.shuffleId
&& mapIndex == o.mapIndex && mapIndex == o.mapIndex
&& reduceId == o.reduceId && reduceId == o.reduceId
@ -85,12 +96,13 @@ public class PushBlockStream extends BlockTransferMessage {
@Override @Override
public int encodedLength() { public int encodedLength() {
return Encoders.Strings.encodedLength(appId) + 16; return Encoders.Strings.encodedLength(appId) + 4 + 4 + 4 + 4 + 4;
} }
@Override @Override
public void encode(ByteBuf buf) { public void encode(ByteBuf buf) {
Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, appId);
buf.writeInt(appAttemptId);
buf.writeInt(shuffleId); buf.writeInt(shuffleId);
buf.writeInt(mapIndex); buf.writeInt(mapIndex);
buf.writeInt(reduceId); buf.writeInt(reduceId);
@ -99,10 +111,11 @@ public class PushBlockStream extends BlockTransferMessage {
public static PushBlockStream decode(ByteBuf buf) { public static PushBlockStream decode(ByteBuf buf) {
String appId = Encoders.Strings.decode(buf); String appId = Encoders.Strings.decode(buf);
int attemptId = buf.readInt();
int shuffleId = buf.readInt(); int shuffleId = buf.readInt();
int mapIdx = buf.readInt(); int mapIdx = buf.readInt();
int reduceId = buf.readInt(); int reduceId = buf.readInt();
int index = buf.readInt(); int index = buf.readInt();
return new PushBlockStream(appId, shuffleId, mapIdx, reduceId, index); return new PushBlockStream(appId, attemptId, shuffleId, mapIdx, reduceId, index);
} }
} }

View file

@ -243,7 +243,7 @@ public class ExternalBlockHandlerSuite {
public void testFinalizeShuffleMerge() throws IOException { public void testFinalizeShuffleMerge() throws IOException {
RpcResponseCallback callback = mock(RpcResponseCallback.class); RpcResponseCallback callback = mock(RpcResponseCallback.class);
FinalizeShuffleMerge req = new FinalizeShuffleMerge("app0", 0); FinalizeShuffleMerge req = new FinalizeShuffleMerge("app0", 1, 0);
RoaringBitmap bitmap = RoaringBitmap.bitmapOf(0, 1, 2); RoaringBitmap bitmap = RoaringBitmap.bitmapOf(0, 1, 2);
MergeStatuses statuses = new MergeStatuses(0, new RoaringBitmap[]{bitmap}, MergeStatuses statuses = new MergeStatuses(0, new RoaringBitmap[]{bitmap},
new int[]{3}, new long[]{30}); new int[]{3}, new long[]{30});

View file

@ -51,7 +51,7 @@ public class OneForOneBlockPusherSuite {
BlockFetchingListener listener = pushBlocks( BlockFetchingListener listener = pushBlocks(
blocks, blocks,
blockIds, blockIds,
Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0))); Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0, 0)));
verify(listener).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); verify(listener).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any());
} }
@ -67,9 +67,9 @@ public class OneForOneBlockPusherSuite {
BlockFetchingListener listener = pushBlocks( BlockFetchingListener listener = pushBlocks(
blocks, blocks,
blockIds, blockIds,
Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), Arrays.asList(new PushBlockStream("app-id",0, 0, 0, 0, 0),
new PushBlockStream("app-id", 0, 1, 0, 1), new PushBlockStream("app-id", 0, 0, 1, 0, 1),
new PushBlockStream("app-id", 0, 2, 0, 2))); new PushBlockStream("app-id", 0, 0, 2, 0, 2)));
verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any());
verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_1_0"), any()); verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_1_0"), any());
@ -87,9 +87,9 @@ public class OneForOneBlockPusherSuite {
BlockFetchingListener listener = pushBlocks( BlockFetchingListener listener = pushBlocks(
blocks, blocks,
blockIds, blockIds,
Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0, 0),
new PushBlockStream("app-id", 0, 1, 0, 1), new PushBlockStream("app-id", 0, 0, 1, 0, 1),
new PushBlockStream("app-id", 0, 2, 0, 2))); new PushBlockStream("app-id", 0, 0, 2, 0, 2)));
verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any());
verify(listener, times(1)).onBlockFetchFailure(eq("shufflePush_0_1_0"), any()); verify(listener, times(1)).onBlockFetchFailure(eq("shufflePush_0_1_0"), any());
@ -107,9 +107,9 @@ public class OneForOneBlockPusherSuite {
BlockFetchingListener listener = pushBlocks( BlockFetchingListener listener = pushBlocks(
blocks, blocks,
blockIds, blockIds,
Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0, 0),
new PushBlockStream("app-id", 0, 1, 0, 1), new PushBlockStream("app-id", 0, 0, 1, 0, 1),
new PushBlockStream("app-id", 0, 2, 0, 2))); new PushBlockStream("app-id", 0, 0, 2, 0, 2)));
verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any());
verify(listener, times(0)).onBlockFetchSuccess(not(eq("shufflePush_0_0_0")), any()); verify(listener, times(0)).onBlockFetchSuccess(not(eq("shufflePush_0_0_0")), any());
@ -130,7 +130,7 @@ public class OneForOneBlockPusherSuite {
TransportClient client = mock(TransportClient.class); TransportClient client = mock(TransportClient.class);
BlockFetchingListener listener = mock(BlockFetchingListener.class); BlockFetchingListener listener = mock(BlockFetchingListener.class);
OneForOneBlockPusher pusher = OneForOneBlockPusher pusher =
new OneForOneBlockPusher(client, "app-id", blockIds, listener, blocks); new OneForOneBlockPusher(client, "app-id", 0, blockIds, listener, blocks);
Iterator<Map.Entry<String, ManagedBuffer>> blockIterator = blocks.entrySet().iterator(); Iterator<Map.Entry<String, ManagedBuffer>> blockIterator = blocks.entrySet().iterator();
Iterator<BlockTransferMessage> msgIterator = expectMessages.iterator(); Iterator<BlockTransferMessage> msgIterator = expectMessages.iterator();

View file

@ -22,11 +22,13 @@ import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.Semaphore; import java.util.concurrent.Semaphore;
import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadLocalRandom;
@ -61,6 +63,17 @@ public class RemoteBlockPushResolverSuite {
private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
private final String TEST_APP = "testApp"; private final String TEST_APP = "testApp";
private final String MERGE_DIRECTORY = "merge_manager";
private final int NO_ATTEMPT_ID = -1;
private final int ATTEMPT_ID_1 = 1;
private final int ATTEMPT_ID_2 = 2;
private final String MERGE_DIRECTORY_META = "shuffleManager:{\"mergeDir\": \"merge_manager\"}";
private final String MERGE_DIRECTORY_META_1 =
"shuffleManager:{\"mergeDir\": \"merge_manager_1\", \"attemptId\": \"1\"}";
private final String MERGE_DIRECTORY_META_2 =
"shuffleManager:{\"mergeDir\": \"merge_manager_2\", \"attemptId\": \"2\"}";
private final String INVALID_MERGE_DIRECTORY_META =
"shuffleManager:{\"mergeDirInvalid\": \"merge_manager_2\", \"attemptId\": \"2\"}";
private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401"; private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
private TransportConf conf; private TransportConf conf;
@ -74,7 +87,7 @@ public class RemoteBlockPushResolverSuite {
ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
conf = new TransportConf("shuffle", provider); conf = new TransportConf("shuffle", provider);
pushResolver = new RemoteBlockPushResolver(conf); pushResolver = new RemoteBlockPushResolver(conf);
registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); registerExecutor(TEST_APP, prepareLocalDirs(localDirs, MERGE_DIRECTORY), MERGE_DIRECTORY_META);
} }
@After @After
@ -106,9 +119,9 @@ public class RemoteBlockPushResolverSuite {
new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4])), new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4])),
new PushBlock(0, 1, 0, ByteBuffer.wrap(new byte[5])) new PushBlock(0, 1, 0, ByteBuffer.wrap(new byte[5]))
}; };
pushBlockHelper(TEST_APP, pushBlocks); pushBlockHelper(TEST_APP, NO_ATTEMPT_ID, pushBlocks);
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {0}, new long[] {9}); validateMergeStatuses(statuses, new int[] {0}, new long[] {9});
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}});
@ -122,9 +135,9 @@ public class RemoteBlockPushResolverSuite {
new PushBlock(0, 2, 0, ByteBuffer.wrap(new byte[5])), new PushBlock(0, 2, 0, ByteBuffer.wrap(new byte[5])),
new PushBlock(0, 3, 0, ByteBuffer.wrap(new byte[3])) new PushBlock(0, 3, 0, ByteBuffer.wrap(new byte[3]))
}; };
pushBlockHelper(TEST_APP, pushBlocks); pushBlockHelper(TEST_APP, NO_ATTEMPT_ID, pushBlocks);
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {0}, new long[] {13}); validateMergeStatuses(statuses, new int[] {0}, new long[] {13});
MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}});
@ -138,9 +151,9 @@ public class RemoteBlockPushResolverSuite {
new PushBlock(0, 0, 1, ByteBuffer.wrap(new byte[5])), new PushBlock(0, 0, 1, ByteBuffer.wrap(new byte[5])),
new PushBlock(0, 1, 1, ByteBuffer.wrap(new byte[3])) new PushBlock(0, 1, 1, ByteBuffer.wrap(new byte[3]))
}; };
pushBlockHelper(TEST_APP, pushBlocks); pushBlockHelper(TEST_APP, NO_ATTEMPT_ID, pushBlocks);
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {0, 1}, new long[] {5, 8}); validateMergeStatuses(statuses, new int[] {0, 1}, new long[] {5, 8});
MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, meta, new int[]{5}, new int[][]{{0, 1}}); validateChunks(TEST_APP, 0, 0, meta, new int[]{5}, new int[][]{{0, 1}});
@ -149,10 +162,12 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testDeferredBufsAreWrittenDuringOnData() throws IOException { public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
// This should be deferred // This should be deferred
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
// stream 1 now completes // stream 1 now completes
@ -161,7 +176,7 @@ public class RemoteBlockPushResolverSuite {
// stream 2 has more data and then completes // stream 2 has more data and then completes
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
} }
@ -169,10 +184,12 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException { public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
// This should be deferred // This should be deferred
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
@ -181,7 +198,7 @@ public class RemoteBlockPushResolverSuite {
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
// stream 2 now completes completes // stream 2 now completes completes
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
} }
@ -189,17 +206,19 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOException { public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
// This should be ignored // This should be ignored
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
} }
@ -207,10 +226,12 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOException { public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
// This should be ignored // This should be ignored
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
@ -219,7 +240,7 @@ public class RemoteBlockPushResolverSuite {
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
// stream 2 now completes completes // stream 2 now completes completes
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
} }
@ -227,10 +248,11 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testFailureAfterData() throws IOException { public void testFailureAfterData() throws IOException {
StreamCallbackWithID stream = StreamCallbackWithID stream =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
stream.onFailure(stream.getID(), new RuntimeException("Forced Failure")); stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
assertEquals("num-chunks", 0, blockMeta.getNumChunks()); assertEquals("num-chunks", 0, blockMeta.getNumChunks());
} }
@ -238,12 +260,13 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testFailureAfterMultipleDataBlocks() throws IOException { public void testFailureAfterMultipleDataBlocks() throws IOException {
StreamCallbackWithID stream = StreamCallbackWithID stream =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
stream.onFailure(stream.getID(), new RuntimeException("Forced Failure")); stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
assertEquals("num-chunks", 0, blockMeta.getNumChunks()); assertEquals("num-chunks", 0, blockMeta.getNumChunks());
} }
@ -251,39 +274,39 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testFailureAfterComplete() throws IOException { public void testFailureAfterComplete() throws IOException {
StreamCallbackWithID stream = StreamCallbackWithID stream =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4])); stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
stream.onComplete(stream.getID()); stream.onComplete(stream.getID());
stream.onFailure(stream.getID(), new RuntimeException("Forced Failure")); stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testTooLateArrival() throws IOException { public void testBlockReceivedAfterMergeFinalize() throws IOException {
ByteBuffer[] blocks = new ByteBuffer[]{ ByteBuffer[] blocks = new ByteBuffer[]{
ByteBuffer.wrap(new byte[4]), ByteBuffer.wrap(new byte[4]),
ByteBuffer.wrap(new byte[5]) ByteBuffer.wrap(new byte[5])
}; };
StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
for (ByteBuffer block : blocks) { for (ByteBuffer block : blocks) {
stream.onData(stream.getID(), block); stream.onData(stream.getID(), block);
} }
stream.onComplete(stream.getID()); stream.onComplete(stream.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 1, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4]));
try { try {
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
} catch (RuntimeException re) { } catch (RuntimeException re) {
assertEquals( assertEquals(
"Block shufflePush_0_1_0 received after merged shuffle is finalized", "Block shufflePush_0_1_0 received after merged shuffle is finalized", re.getMessage());
re.getMessage());
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
throw re; throw re;
@ -292,28 +315,31 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testIncompleteStreamsAreOverwritten() throws IOException { public void testIncompleteStreamsAreOverwritten() throws IOException {
registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); registerExecutor(TEST_APP, prepareLocalDirs(localDirs, MERGE_DIRECTORY), MERGE_DIRECTORY_META);
byte[] expectedBytes = new byte[4]; byte[] expectedBytes = new byte[4];
ThreadLocalRandom.current().nextBytes(expectedBytes); ThreadLocalRandom.current().nextBytes(expectedBytes);
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
byte[] data = new byte[10]; byte[] data = new byte[10];
ThreadLocalRandom.current().nextBytes(data); ThreadLocalRandom.current().nextBytes(data);
stream1.onData(stream1.getID(), ByteBuffer.wrap(data)); stream1.onData(stream1.getID(), ByteBuffer.wrap(data));
// There is a failure // There is a failure
stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); stream1.onFailure(stream1.getID(), new RuntimeException("forced error"));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
ByteBuffer nextBuf= ByteBuffer.wrap(expectedBytes, 0, 2); ByteBuffer nextBuf= ByteBuffer.wrap(expectedBytes, 0, 2);
stream2.onData(stream2.getID(), nextBuf); stream2.onData(stream2.getID(), nextBuf);
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
StreamCallbackWithID stream3 = StreamCallbackWithID stream3 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
nextBuf = ByteBuffer.wrap(expectedBytes, 2, 2); nextBuf = ByteBuffer.wrap(expectedBytes, 2, 2);
stream3.onData(stream3.getID(), nextBuf); stream3.onData(stream3.getID(), nextBuf);
stream3.onComplete(stream3.getID()); stream3.onComplete(stream3.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{1, 2}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{1, 2}});
FileSegmentManagedBuffer mb = FileSegmentManagedBuffer mb =
@ -321,13 +347,15 @@ public class RemoteBlockPushResolverSuite {
assertArrayEquals(expectedBytes, mb.nioByteBuffer().array()); assertArrayEquals(expectedBytes, mb.nioByteBuffer().array());
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testCollision() throws IOException { public void testCollision() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
// This should be deferred // This should be deferred
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5]));
// Since stream2 didn't get any opportunity it will throw couldn't find opportunity error // Since stream2 didn't get any opportunity it will throw couldn't find opportunity error
@ -341,17 +369,20 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testFailureInAStreamDoesNotInterfereWithStreamWhichIsWriting() throws IOException { public void testFailureInAStreamDoesNotInterfereWithStreamWhichIsWriting() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
// There is a failure with stream2 // There is a failure with stream2
stream2.onFailure(stream2.getID(), new RuntimeException("forced error")); stream2.onFailure(stream2.getID(), new RuntimeException("forced error"));
StreamCallbackWithID stream3 = StreamCallbackWithID stream3 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
// This should be deferred // This should be deferred
stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[5])); stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[5]));
// Since this stream didn't get any opportunity it will throw couldn't find opportunity error // Since this stream didn't get any opportunity it will throw couldn't find opportunity error
@ -368,7 +399,7 @@ public class RemoteBlockPushResolverSuite {
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4}, new int[][] {{0}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4}, new int[][] {{0}});
if (failedEx != null) { if (failedEx != null) {
@ -376,28 +407,83 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test(expected = NullPointerException.class) @Test(expected = IllegalArgumentException.class)
public void testUpdateLocalDirsOnlyOnce() throws IOException { public void testUpdateLocalDirsOnlyOnce() throws IOException {
String testApp = "updateLocalDirsOnlyOnceTest"; String testApp = "updateLocalDirsOnlyOnceTest";
Path[] activeLocalDirs = createLocalDirs(1); Path[] activeLocalDirs = createLocalDirs(1);
registerExecutor(testApp, prepareLocalDirs(activeLocalDirs)); registerExecutor(testApp, prepareLocalDirs(activeLocalDirs, MERGE_DIRECTORY),
MERGE_DIRECTORY_META);
assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1); assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains( assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
activeLocalDirs[0].toFile().getPath())); activeLocalDirs[0].toFile().getPath()));
// Any later executor register from the same application should not change the active local // Any later executor register from the same application attempt should not change the active
// dirs list // local dirs list
Path[] updatedLocalDirs = localDirs; Path[] updatedLocalDirs = localDirs;
registerExecutor(testApp, prepareLocalDirs(updatedLocalDirs)); registerExecutor(testApp, prepareLocalDirs(updatedLocalDirs, MERGE_DIRECTORY),
MERGE_DIRECTORY_META);
assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1); assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains( assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
activeLocalDirs[0].toFile().getPath())); activeLocalDirs[0].toFile().getPath()));
removeApplication(testApp); removeApplication(testApp);
try { try {
pushResolver.getMergedBlockDirs(testApp); pushResolver.getMergedBlockDirs(testApp);
} catch (Throwable e) { } catch (IllegalArgumentException e) {
assertTrue(e.getMessage() assertEquals(e.getMessage(),
.startsWith("application " + testApp + " is not registered or NM was restarted.")); "application " + testApp + " is not registered or NM was restarted.");
Throwables.propagate(e); throw e;
}
}
@Test(expected = IllegalArgumentException.class)
public void testExecutorRegisterWithInvalidJsonForPushShuffle() throws IOException {
String testApp = "executorRegisterWithInvalidShuffleManagerMeta";
Path[] activeLocalDirs = createLocalDirs(1);
try {
registerExecutor(testApp, prepareLocalDirs(activeLocalDirs, MERGE_DIRECTORY),
INVALID_MERGE_DIRECTORY_META);
} catch (IllegalArgumentException re) {
assertEquals(
"Failed to get the merge directory information from the shuffleManagerMeta " +
"shuffleManager:{\"mergeDirInvalid\": \"merge_manager_2\", \"attemptId\": \"2\"} in " +
"executor registration message", re.getMessage());
throw re;
}
}
@Test(expected = IllegalArgumentException.class)
public void testExecutorRegistrationFromTwoAppAttempts() throws IOException {
String testApp = "testExecutorRegistrationFromTwoAppAttempts";
Path[] attempt1LocalDirs = createLocalDirs(1);
registerExecutor(testApp,
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
MERGE_DIRECTORY_META_1);
assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
attempt1LocalDirs[0].toFile().getPath()));
// Any later executor register from the same application attempt should not change the active
// local dirs list
Path[] attempt1UpdatedLocalDirs = localDirs;
registerExecutor(testApp,
prepareLocalDirs(attempt1UpdatedLocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
MERGE_DIRECTORY_META_1);
assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
attempt1LocalDirs[0].toFile().getPath()));
// But a new attempt from the same application can change the active local dirs list
Path[] attempt2LocalDirs = createLocalDirs(2);
registerExecutor(testApp,
prepareLocalDirs(attempt2LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_2),
MERGE_DIRECTORY_META_2);
assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 2);
assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
attempt2LocalDirs[0].toFile().getPath()));
removeApplication(testApp);
try {
pushResolver.getMergedBlockDirs(testApp);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(),
"application " + testApp + " is not registered or NM was restarted.");
throw e;
} }
} }
@ -407,17 +493,18 @@ public class RemoteBlockPushResolverSuite {
Semaphore deleted = new Semaphore(0); Semaphore deleted = new Semaphore(0);
pushResolver = new RemoteBlockPushResolver(conf) { pushResolver = new RemoteBlockPushResolver(conf) {
@Override @Override
void deleteExecutorDirs(Path[] dirs) { void deleteExecutorDirs(AppShuffleInfo appShuffleInfo) {
super.deleteExecutorDirs(dirs); super.deleteExecutorDirs(appShuffleInfo);
deleted.release(); deleted.release();
} }
}; };
Path[] activeDirs = createLocalDirs(1); Path[] activeDirs = createLocalDirs(1);
registerExecutor(testApp, prepareLocalDirs(activeDirs)); registerExecutor(testApp, prepareLocalDirs(activeDirs, MERGE_DIRECTORY), MERGE_DIRECTORY_META);
PushBlock[] pushBlocks = new PushBlock[] { PushBlock[] pushBlocks = new PushBlock[] {
new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4]))}; new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4]))};
pushBlockHelper(testApp, pushBlocks); pushBlockHelper(testApp, NO_ATTEMPT_ID, pushBlocks);
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(testApp, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(testApp, 0, 0);
validateChunks(testApp, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}}); validateChunks(testApp, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
String[] mergeDirs = pushResolver.getMergedBlockDirs(testApp); String[] mergeDirs = pushResolver.getMergedBlockDirs(testApp);
@ -435,7 +522,7 @@ public class RemoteBlockPushResolverSuite {
useTestFiles(true, false); useTestFiles(true, false);
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4]));
callback1.onComplete(callback1.getID()); callback1.onComplete(callback1.getID());
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo();
@ -443,7 +530,7 @@ public class RemoteBlockPushResolverSuite {
TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile(); TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile();
testIndexFile.close(); testIndexFile.close();
StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 1, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any IOExceptions because number of IOExceptions are less than // This will complete without any IOExceptions because number of IOExceptions are less than
// the threshold but the update to index file will be unsuccessful. // the threshold but the update to index file will be unsuccessful.
@ -452,12 +539,12 @@ public class RemoteBlockPushResolverSuite {
// Restore the index stream so it can write successfully again. // Restore the index stream so it can write successfully again.
testIndexFile.restore(); testIndexFile.restore();
StreamCallbackWithID callback3 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback3 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 2, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
callback3.onData(callback3.getID(), ByteBuffer.wrap(new byte[2])); callback3.onData(callback3.getID(), ByteBuffer.wrap(new byte[2]));
callback3.onComplete(callback3.getID()); callback3.onComplete(callback3.getID());
assertEquals("index position", 24, testIndexFile.getPos()); assertEquals("index position", 24, testIndexFile.getPos());
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {0}, new long[] {11}); validateMergeStatuses(statuses, new int[] {0}, new long[] {11});
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 7}, new int[][] {{0}, {1, 2}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 7}, new int[][] {{0}, {1, 2}});
@ -468,7 +555,7 @@ public class RemoteBlockPushResolverSuite {
useTestFiles(true, false); useTestFiles(true, false);
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4]));
callback1.onComplete(callback1.getID()); callback1.onComplete(callback1.getID());
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo();
@ -476,7 +563,7 @@ public class RemoteBlockPushResolverSuite {
TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile(); TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile();
testIndexFile.close(); testIndexFile.close();
StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 1, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any IOExceptions because number of IOExceptions are less than // This will complete without any IOExceptions because number of IOExceptions are less than
// the threshold but the update to index file will be unsuccessful. // the threshold but the update to index file will be unsuccessful.
@ -486,7 +573,7 @@ public class RemoteBlockPushResolverSuite {
// Restore the index stream so it can write successfully again. // Restore the index stream so it can write successfully again.
testIndexFile.restore(); testIndexFile.restore();
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
assertEquals("index position", 24, testIndexFile.getPos()); assertEquals("index position", 24, testIndexFile.getPos());
validateMergeStatuses(statuses, new int[] {0}, new long[] {9}); validateMergeStatuses(statuses, new int[] {0}, new long[] {9});
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
@ -498,7 +585,7 @@ public class RemoteBlockPushResolverSuite {
useTestFiles(false, true); useTestFiles(false, true);
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4]));
callback1.onComplete(callback1.getID()); callback1.onComplete(callback1.getID());
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo();
@ -507,7 +594,7 @@ public class RemoteBlockPushResolverSuite {
long metaPosBeforeClose = testMetaFile.getPos(); long metaPosBeforeClose = testMetaFile.getPos();
testMetaFile.close(); testMetaFile.close();
StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 1, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any IOExceptions because number of IOExceptions are less than // This will complete without any IOExceptions because number of IOExceptions are less than
// the threshold but the update to index and meta file will be unsuccessful. // the threshold but the update to index and meta file will be unsuccessful.
@ -517,13 +604,13 @@ public class RemoteBlockPushResolverSuite {
// Restore the meta stream so it can write successfully again. // Restore the meta stream so it can write successfully again.
testMetaFile.restore(); testMetaFile.restore();
StreamCallbackWithID callback3 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback3 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 2, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
callback3.onData(callback3.getID(), ByteBuffer.wrap(new byte[2])); callback3.onData(callback3.getID(), ByteBuffer.wrap(new byte[2]));
callback3.onComplete(callback3.getID()); callback3.onComplete(callback3.getID());
assertEquals("index position", 24, partitionInfo.getIndexFile().getPos()); assertEquals("index position", 24, partitionInfo.getIndexFile().getPos());
assertTrue("meta position", testMetaFile.getPos() > metaPosBeforeClose); assertTrue("meta position", testMetaFile.getPos() > metaPosBeforeClose);
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {0}, new long[] {11}); validateMergeStatuses(statuses, new int[] {0}, new long[] {11});
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 7}, new int[][] {{0}, {1, 2}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 7}, new int[][] {{0}, {1, 2}});
@ -534,7 +621,7 @@ public class RemoteBlockPushResolverSuite {
useTestFiles(false, true); useTestFiles(false, true);
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[4]));
callback1.onComplete(callback1.getID()); callback1.onComplete(callback1.getID());
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback1.getPartitionInfo();
@ -543,7 +630,7 @@ public class RemoteBlockPushResolverSuite {
long metaPosBeforeClose = testMetaFile.getPos(); long metaPosBeforeClose = testMetaFile.getPos();
testMetaFile.close(); testMetaFile.close();
StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID callback2 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 1, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any IOExceptions because number of IOExceptions are less than // This will complete without any IOExceptions because number of IOExceptions are less than
// the threshold but the update to index and meta file will be unsuccessful. // the threshold but the update to index and meta file will be unsuccessful.
@ -554,7 +641,7 @@ public class RemoteBlockPushResolverSuite {
// Restore the meta stream so it can write successfully again. // Restore the meta stream so it can write successfully again.
testMetaFile.restore(); testMetaFile.restore();
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
assertEquals("index position", 24, indexFile.getPos()); assertEquals("index position", 24, indexFile.getPos());
assertTrue("meta position", testMetaFile.getPos() > metaPosBeforeClose); assertTrue("meta position", testMetaFile.getPos() > metaPosBeforeClose);
validateMergeStatuses(statuses, new int[] {0}, new long[] {9}); validateMergeStatuses(statuses, new int[] {0}, new long[] {9});
@ -562,11 +649,11 @@ public class RemoteBlockPushResolverSuite {
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 5}, new int[][] {{0}, {1}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4, 5}, new int[][] {{0}, {1}});
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testIOExceptionsExceededThreshold() throws IOException { public void testIOExceptionsExceededThreshold() throws IOException {
RemoteBlockPushResolver.PushBlockStreamCallback callback = RemoteBlockPushResolver.PushBlockStreamCallback callback =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo();
callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4])); callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4]));
callback.onComplete(callback.getID()); callback.onComplete(callback.getID());
@ -575,7 +662,7 @@ public class RemoteBlockPushResolverSuite {
for (int i = 1; i < 5; i++) { for (int i = 1; i < 5; i++) {
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, i, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, i, 0, 0));
try { try {
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[2])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[2]));
} catch (IOException ioe) { } catch (IOException ioe) {
@ -588,7 +675,7 @@ public class RemoteBlockPushResolverSuite {
try { try {
RemoteBlockPushResolver.PushBlockStreamCallback callback2 = RemoteBlockPushResolver.PushBlockStreamCallback callback2 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 5, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 5, 0, 0));
callback2.onData(callback.getID(), ByteBuffer.wrap(new byte[1])); callback2.onData(callback.getID(), ByteBuffer.wrap(new byte[1]));
} catch (Throwable t) { } catch (Throwable t) {
assertEquals("IOExceptions exceeded the threshold when merging shufflePush_0_5_0", assertEquals("IOExceptions exceeded the threshold when merging shufflePush_0_5_0",
@ -597,12 +684,12 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testIOExceptionsDuringMetaUpdateIncreasesExceptionCount() throws IOException { public void testIOExceptionsDuringMetaUpdateIncreasesExceptionCount() throws IOException {
useTestFiles(true, false); useTestFiles(true, false);
RemoteBlockPushResolver.PushBlockStreamCallback callback = RemoteBlockPushResolver.PushBlockStreamCallback callback =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo();
callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4])); callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4]));
callback.onComplete(callback.getID()); callback.onComplete(callback.getID());
@ -611,7 +698,7 @@ public class RemoteBlockPushResolverSuite {
for (int i = 1; i < 5; i++) { for (int i = 1; i < 5; i++) {
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, i, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, i, 0, 0));
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any exceptions but the exception count is increased. // This will complete without any exceptions but the exception count is increased.
callback1.onComplete(callback1.getID()); callback1.onComplete(callback1.getID());
@ -622,7 +709,7 @@ public class RemoteBlockPushResolverSuite {
try { try {
RemoteBlockPushResolver.PushBlockStreamCallback callback2 = RemoteBlockPushResolver.PushBlockStreamCallback callback2 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 5, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 5, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[4])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[4]));
callback2.onComplete(callback2.getID()); callback2.onComplete(callback2.getID());
} catch (Throwable t) { } catch (Throwable t) {
@ -632,7 +719,7 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testRequestForAbortedShufflePartitionThrowsException() { public void testRequestForAbortedShufflePartitionThrowsException() {
try { try {
testIOExceptionsDuringMetaUpdateIncreasesExceptionCount(); testIOExceptionsDuringMetaUpdateIncreasesExceptionCount();
@ -641,7 +728,7 @@ public class RemoteBlockPushResolverSuite {
} }
try { try {
pushResolver.receiveBlockDataAsStream( pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 10, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 10, 0, 0));
} catch (Throwable t) { } catch (Throwable t) {
assertEquals("IOExceptions exceeded the threshold when merging shufflePush_0_10_0", assertEquals("IOExceptions exceeded the threshold when merging shufflePush_0_10_0",
t.getMessage()); t.getMessage());
@ -649,19 +736,19 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testPendingBlockIsAbortedImmediately() throws IOException { public void testPendingBlockIsAbortedImmediately() throws IOException {
useTestFiles(true, false); useTestFiles(true, false);
RemoteBlockPushResolver.PushBlockStreamCallback callback = RemoteBlockPushResolver.PushBlockStreamCallback callback =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo();
TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile(); TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile();
testIndexFile.close(); testIndexFile.close();
for (int i = 1; i < 6; i++) { for (int i = 1; i < 6; i++) {
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, i, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, i, 0, 0));
try { try {
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any exceptions but the exception count is increased. // This will complete without any exceptions but the exception count is increased.
@ -682,19 +769,19 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testWritingPendingBufsIsAbortedImmediatelyDuringComplete() throws IOException { public void testWritingPendingBufsIsAbortedImmediatelyDuringComplete() throws IOException {
useTestFiles(true, false); useTestFiles(true, false);
RemoteBlockPushResolver.PushBlockStreamCallback callback = RemoteBlockPushResolver.PushBlockStreamCallback callback =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 0, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo();
TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile(); TestMergeShuffleFile testIndexFile = (TestMergeShuffleFile) partitionInfo.getIndexFile();
testIndexFile.close(); testIndexFile.close();
for (int i = 1; i < 5; i++) { for (int i = 1; i < 5; i++) {
RemoteBlockPushResolver.PushBlockStreamCallback callback1 = RemoteBlockPushResolver.PushBlockStreamCallback callback1 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, i, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, i, 0, 0));
try { try {
callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5])); callback1.onData(callback1.getID(), ByteBuffer.wrap(new byte[5]));
// This will complete without any exceptions but the exception count is increased. // This will complete without any exceptions but the exception count is increased.
@ -706,7 +793,7 @@ public class RemoteBlockPushResolverSuite {
assertEquals(4, partitionInfo.getNumIOExceptions()); assertEquals(4, partitionInfo.getNumIOExceptions());
RemoteBlockPushResolver.PushBlockStreamCallback callback2 = RemoteBlockPushResolver.PushBlockStreamCallback callback2 =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 5, 0, 0)); new PushBlockStream(TEST_APP, 1, 0, 5, 0, 0));
callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5])); callback2.onData(callback2.getID(), ByteBuffer.wrap(new byte[5]));
// This is deferred // This is deferred
callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4])); callback.onData(callback.getID(), ByteBuffer.wrap(new byte[4]));
@ -738,10 +825,10 @@ public class RemoteBlockPushResolverSuite {
new PushBlock(0, 0, 1, ByteBuffer.wrap(new byte[5])), new PushBlock(0, 0, 1, ByteBuffer.wrap(new byte[5])),
new PushBlock(0, 1, 1, ByteBuffer.wrap(new byte[3])) new PushBlock(0, 1, 1, ByteBuffer.wrap(new byte[3]))
}; };
pushBlockHelper(TEST_APP, pushBlocks); pushBlockHelper(TEST_APP, NO_ATTEMPT_ID, pushBlocks);
RemoteBlockPushResolver.PushBlockStreamCallback callback = RemoteBlockPushResolver.PushBlockStreamCallback callback =
(RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream( (RemoteBlockPushResolver.PushBlockStreamCallback) pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, 0, 2, 0, 0)); new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
callback.onData(callback.getID(), ByteBuffer.wrap(new byte[2])); callback.onData(callback.getID(), ByteBuffer.wrap(new byte[2]));
callback.onComplete(callback.getID()); callback.onComplete(callback.getID());
RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo(); RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo = callback.getPartitionInfo();
@ -749,7 +836,7 @@ public class RemoteBlockPushResolverSuite {
// Close the index file so truncate throws IOException // Close the index file so truncate throws IOException
testIndexFile.close(); testIndexFile.close();
MergeStatuses statuses = pushResolver.finalizeShuffleMerge( MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
new FinalizeShuffleMerge(TEST_APP, 0)); new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
validateMergeStatuses(statuses, new int[] {1}, new long[] {8}); validateMergeStatuses(statuses, new int[] {1}, new long[] {8});
MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 1); MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 1);
validateChunks(TEST_APP, 0, 1, meta, new int[]{5, 3}, new int[][]{{0},{1}}); validateChunks(TEST_APP, 0, 1, meta, new int[]{5, 3}, new int[][]{{0},{1}});
@ -758,46 +845,53 @@ public class RemoteBlockPushResolverSuite {
@Test @Test
public void testOnFailureInvokedMoreThanOncePerBlock() throws IOException { public void testOnFailureInvokedMoreThanOncePerBlock() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); stream1.onFailure(stream1.getID(), new RuntimeException("forced error"));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5]));
// On failure on stream1 gets invoked again and should cause no interference // On failure on stream1 gets invoked again and should cause no interference
stream1.onFailure(stream1.getID(), new RuntimeException("2nd forced error")); stream1.onFailure(stream1.getID(), new RuntimeException("2nd forced error"));
StreamCallbackWithID stream3 = StreamCallbackWithID stream3 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 3, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 3, 0, 0));
// This should be deferred as stream 2 is still the active stream // This should be deferred as stream 2 is still the active stream
stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2]));
// Stream 2 writes more and completes // Stream 2 writes more and completes
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4]));
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
stream3.onComplete(stream3.getID()); stream3.onComplete(stream3.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {9, 2}, new int[][] {{1},{3}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {9, 2}, new int[][] {{1},{3}});
removeApplication(TEST_APP); removeApplication(TEST_APP);
} }
@Test (expected = RuntimeException.class) @Test(expected = RuntimeException.class)
public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws IOException { public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws IOException {
StreamCallbackWithID stream1 = StreamCallbackWithID stream1 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
StreamCallbackWithID stream1Duplicate = StreamCallbackWithID stream1Duplicate =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 0, 0, 0));
stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
stream1.onComplete(stream1.getID()); stream1.onComplete(stream1.getID());
stream1Duplicate.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); stream1Duplicate.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
StreamCallbackWithID stream2 = StreamCallbackWithID stream2 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 1, 0, 0));
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5]));
// Should not change the current map id of the reduce partition // Should not change the current map id of the reduce partition
stream1Duplicate.onFailure(stream2.getID(), new RuntimeException("forced error")); stream1Duplicate.onFailure(stream2.getID(), new RuntimeException("forced error"));
StreamCallbackWithID stream3 = StreamCallbackWithID stream3 =
pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); pushResolver.receiveBlockDataAsStream(
new PushBlockStream(TEST_APP, NO_ATTEMPT_ID, 0, 2, 0, 0));
// This should be deferred as stream 2 is still the active stream // This should be deferred as stream 2 is still the active stream
stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2]));
RuntimeException failedEx = null; RuntimeException failedEx = null;
@ -812,7 +906,7 @@ public class RemoteBlockPushResolverSuite {
// Stream 2 writes more and completes // Stream 2 writes more and completes
stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4]));
stream2.onComplete(stream2.getID()); stream2.onComplete(stream2.getID());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, NO_ATTEMPT_ID, 0));
MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {11}, new int[][] {{0, 1}}); validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {11}, new int[][] {{0, 1}});
removeApplication(TEST_APP); removeApplication(TEST_APP);
@ -821,20 +915,165 @@ public class RemoteBlockPushResolverSuite {
} }
} }
@Test(expected = IllegalArgumentException.class)
public void testPushBlockFromPreviousAttemptIsRejected()
throws IOException, InterruptedException {
Semaphore closed = new Semaphore(0);
pushResolver = new RemoteBlockPushResolver(conf) {
@Override
void closeAndDeletePartitionFilesIfNeeded(
AppShuffleInfo appShuffleInfo,
boolean cleanupLocalDirs) {
super.closeAndDeletePartitionFilesIfNeeded(appShuffleInfo, cleanupLocalDirs);
closed.release();
}
};
String testApp = "testPushBlockFromPreviousAttemptIsRejected";
Path[] attempt1LocalDirs = createLocalDirs(1);
registerExecutor(testApp,
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
MERGE_DIRECTORY_META_1);
ByteBuffer[] blocks = new ByteBuffer[]{
ByteBuffer.wrap(new byte[4]),
ByteBuffer.wrap(new byte[5])
};
StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(testApp, 1, 0, 0, 0, 0));
for (ByteBuffer block : blocks) {
stream1.onData(stream1.getID(), block);
}
stream1.onComplete(stream1.getID());
RemoteBlockPushResolver.AppShuffleInfo appShuffleInfo =
pushResolver.validateAndGetAppShuffleInfo(testApp);
Map<Integer, Map<Integer, RemoteBlockPushResolver.AppShufflePartitionInfo>> partitions =
appShuffleInfo.getPartitions();
for (Map<Integer, RemoteBlockPushResolver.AppShufflePartitionInfo> partitionMap :
partitions.values()) {
for (RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo : partitionMap.values()) {
assertTrue(partitionInfo.getDataChannel().isOpen());
assertTrue(partitionInfo.getMetaFile().getChannel().isOpen());
assertTrue(partitionInfo.getIndexFile().getChannel().isOpen());
}
}
Path[] attempt2LocalDirs = createLocalDirs(2);
registerExecutor(testApp,
prepareLocalDirs(attempt2LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_2),
MERGE_DIRECTORY_META_2);
StreamCallbackWithID stream2 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(testApp, 2, 0, 1, 0, 0));
for (ByteBuffer block : blocks) {
stream2.onData(stream2.getID(), block);
}
stream2.onComplete(stream2.getID());
closed.acquire();
// Check if all the file channels created for the first attempt are safely closed.
for (Map<Integer, RemoteBlockPushResolver.AppShufflePartitionInfo> partitionMap :
partitions.values()) {
for (RemoteBlockPushResolver.AppShufflePartitionInfo partitionInfo : partitionMap.values()) {
assertFalse(partitionInfo.getDataChannel().isOpen());
assertFalse(partitionInfo.getMetaFile().getChannel().isOpen());
assertFalse(partitionInfo.getIndexFile().getChannel().isOpen());
}
}
try {
pushResolver.receiveBlockDataAsStream(
new PushBlockStream(testApp, 1, 0, 1, 0, 0));
} catch (IllegalArgumentException re) {
assertEquals(
"The attempt id 1 in this PushBlockStream message does not match " +
"with the current attempt id 2 stored in shuffle service for application " +
testApp, re.getMessage());
throw re;
}
}
@Test(expected = IllegalArgumentException.class)
public void testFinalizeShuffleMergeFromPreviousAttemptIsAborted()
throws IOException, InterruptedException {
String testApp = "testFinalizeShuffleMergeFromPreviousAttemptIsAborted";
Path[] attempt1LocalDirs = createLocalDirs(1);
registerExecutor(testApp,
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
MERGE_DIRECTORY_META_1);
ByteBuffer[] blocks = new ByteBuffer[]{
ByteBuffer.wrap(new byte[4]),
ByteBuffer.wrap(new byte[5])
};
StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(testApp, 1, 0, 0, 0, 0));
for (ByteBuffer block : blocks) {
stream1.onData(stream1.getID(), block);
}
stream1.onComplete(stream1.getID());
Path[] attempt2LocalDirs = createLocalDirs(2);
registerExecutor(testApp,
prepareLocalDirs(attempt2LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_2),
MERGE_DIRECTORY_META_2);
try {
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, ATTEMPT_ID_1, 0));
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(),
String.format("The attempt id %s in this FinalizeShuffleMerge message does not " +
"match with the current attempt id %s stored in shuffle service for application %s",
ATTEMPT_ID_1, ATTEMPT_ID_2, testApp));
throw e;
}
}
@Test(expected = ClosedChannelException.class)
public void testOngoingMergeOfBlockFromPreviousAttemptIsAborted()
throws IOException, InterruptedException {
Semaphore closed = new Semaphore(0);
pushResolver = new RemoteBlockPushResolver(conf) {
@Override
void closeAndDeletePartitionFilesIfNeeded(
AppShuffleInfo appShuffleInfo,
boolean cleanupLocalDirs) {
super.closeAndDeletePartitionFilesIfNeeded(appShuffleInfo, cleanupLocalDirs);
closed.release();
}
};
String testApp = "testOngoingMergeOfBlockFromPreviousAttemptIsAborted";
Path[] attempt1LocalDirs = createLocalDirs(1);
registerExecutor(testApp,
prepareLocalDirs(attempt1LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_1),
MERGE_DIRECTORY_META_1);
ByteBuffer[] blocks = new ByteBuffer[]{
ByteBuffer.wrap(new byte[4]),
ByteBuffer.wrap(new byte[5]),
ByteBuffer.wrap(new byte[6]),
ByteBuffer.wrap(new byte[7])
};
StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(testApp, 1, 0, 0, 0, 0));
// The onData callback should be called 4 times here before the onComplete callback. But a
// register executor message arrives in shuffle service after the 2nd onData callback. The 3rd
// onData callback should all throw ClosedChannelException as their channels are closed.
stream1.onData(stream1.getID(), blocks[0]);
stream1.onData(stream1.getID(), blocks[1]);
Path[] attempt2LocalDirs = createLocalDirs(2);
registerExecutor(testApp,
prepareLocalDirs(attempt2LocalDirs, MERGE_DIRECTORY + "_" + ATTEMPT_ID_2),
MERGE_DIRECTORY_META_2);
closed.acquire();
// Should throw ClosedChannelException here.
stream1.onData(stream1.getID(), blocks[3]);
}
private void useTestFiles(boolean useTestIndexFile, boolean useTestMetaFile) throws IOException { private void useTestFiles(boolean useTestIndexFile, boolean useTestMetaFile) throws IOException {
pushResolver = new RemoteBlockPushResolver(conf) { pushResolver = new RemoteBlockPushResolver(conf) {
@Override @Override
AppShufflePartitionInfo newAppShufflePartitionInfo(AppShuffleId appShuffleId, int reduceId, AppShufflePartitionInfo newAppShufflePartitionInfo(String appId, int shuffleId,
File dataFile, File indexFile, File metaFile) throws IOException { int reduceId, File dataFile, File indexFile, File metaFile) throws IOException {
MergeShuffleFile mergedIndexFile = useTestIndexFile ? new TestMergeShuffleFile(indexFile) MergeShuffleFile mergedIndexFile = useTestIndexFile ? new TestMergeShuffleFile(indexFile)
: new MergeShuffleFile(indexFile); : new MergeShuffleFile(indexFile);
MergeShuffleFile mergedMetaFile = useTestMetaFile ? new TestMergeShuffleFile(metaFile) : MergeShuffleFile mergedMetaFile = useTestMetaFile ? new TestMergeShuffleFile(metaFile) :
new MergeShuffleFile(metaFile); new MergeShuffleFile(metaFile);
return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, mergedIndexFile, return new AppShufflePartitionInfo(appId, shuffleId, reduceId, dataFile, mergedIndexFile,
mergedMetaFile); mergedMetaFile);
} }
}; };
registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); registerExecutor(TEST_APP, prepareLocalDirs(localDirs, MERGE_DIRECTORY), MERGE_DIRECTORY_META);
} }
private Path[] createLocalDirs(int numLocalDirs) throws IOException { private Path[] createLocalDirs(int numLocalDirs) throws IOException {
@ -846,16 +1085,15 @@ public class RemoteBlockPushResolverSuite {
return localDirs; return localDirs;
} }
private void registerExecutor(String appId, String[] localDirs) throws IOException { private void registerExecutor(String appId, String[] localDirs, String shuffleManagerMeta) {
ExecutorShuffleInfo shuffleInfo = new ExecutorShuffleInfo(localDirs, 1, "mergedShuffle"); ExecutorShuffleInfo shuffleInfo = new ExecutorShuffleInfo(localDirs, 1, shuffleManagerMeta);
pushResolver.registerExecutor(appId, shuffleInfo); pushResolver.registerExecutor(appId, shuffleInfo);
} }
private String[] prepareLocalDirs(Path[] localDirs) throws IOException { private String[] prepareLocalDirs(Path[] localDirs, String mergeDir) throws IOException {
String[] blockMgrDirs = new String[localDirs.length]; String[] blockMgrDirs = new String[localDirs.length];
for (int i = 0; i< localDirs.length; i++) { for (int i = 0; i< localDirs.length; i++) {
Files.createDirectories(localDirs[i].resolve( Files.createDirectories(localDirs[i].resolve(mergeDir + File.separator + "00"));
RemoteBlockPushResolver.MERGE_MANAGER_DIR + File.separator + "00"));
blockMgrDirs[i] = localDirs[i].toFile().getPath() + File.separator + BLOCK_MANAGER_DIR; blockMgrDirs[i] = localDirs[i].toFile().getPath() + File.separator + BLOCK_MANAGER_DIR;
} }
return blockMgrDirs; return blockMgrDirs;
@ -898,10 +1136,12 @@ public class RemoteBlockPushResolverSuite {
private void pushBlockHelper( private void pushBlockHelper(
String appId, String appId,
int attemptId,
PushBlock[] blocks) throws IOException { PushBlock[] blocks) throws IOException {
for (int i = 0; i < blocks.length; i++) { for (int i = 0; i < blocks.length; i++) {
StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
new PushBlockStream(appId, blocks[i].shuffleId, blocks[i].mapIndex, blocks[i].reduceId, 0)); new PushBlockStream(
appId, attemptId, blocks[i].shuffleId, blocks[i].mapIndex, blocks[i].reduceId, 0));
stream.onData(stream.getID(), blocks[i].buffer); stream.onData(stream.getID(), blocks[i].buffer);
stream.onComplete(stream.getID()); stream.onComplete(stream.getID());
} }

View file

@ -583,6 +583,7 @@ class SparkContext(config: SparkConf) extends Logging {
_applicationId = _taskScheduler.applicationId() _applicationId = _taskScheduler.applicationId()
_applicationAttemptId = _taskScheduler.applicationAttemptId() _applicationAttemptId = _taskScheduler.applicationAttemptId()
_conf.set("spark.app.id", _applicationId) _conf.set("spark.app.id", _applicationId)
_applicationAttemptId.foreach(attemptId => _conf.set(APP_ATTEMPT_ID, attemptId))
if (_conf.get(UI_REVERSE_PROXY)) { if (_conf.get(UI_REVERSE_PROXY)) {
val proxyUrl = _conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/") + val proxyUrl = _conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/") +
"/proxy/" + _applicationId "/proxy/" + _applicationId

View file

@ -2244,4 +2244,14 @@ package object config {
.stringConf .stringConf
.toSequence .toSequence
.createWithDefault(Nil) .createWithDefault(Nil)
private[spark] val APP_ATTEMPT_ID =
ConfigBuilder("spark.app.attempt.id")
.internal()
.doc("The application attempt Id assigned from Hadoop YARN. " +
"When the application runs in cluster mode on YARN, there can be " +
"multiple attempts before failing the application")
.version("3.2.0")
.stringConf
.createOptional
} }

View file

@ -535,10 +535,17 @@ private[spark] class BlockManager(
private def registerWithExternalShuffleServer(): Unit = { private def registerWithExternalShuffleServer(): Unit = {
logInfo("Registering executor with local external shuffle service.") logInfo("Registering executor with local external shuffle service.")
val shuffleManagerMeta =
if (Utils.isPushBasedShuffleEnabled(conf)) {
s"${shuffleManager.getClass.getName}:" +
s"${diskBlockManager.getMergeDirectoryAndAttemptIDJsonString()}}}"
} else {
shuffleManager.getClass.getName
}
val shuffleConfig = new ExecutorShuffleInfo( val shuffleConfig = new ExecutorShuffleInfo(
diskBlockManager.localDirsString, diskBlockManager.localDirsString,
diskBlockManager.subDirsPerLocalDir, diskBlockManager.subDirsPerLocalDir,
shuffleManager.getClass.getName) shuffleManagerMeta)
val MAX_ATTEMPTS = conf.get(config.SHUFFLE_REGISTRATION_MAX_ATTEMPTS) val MAX_ATTEMPTS = conf.get(config.SHUFFLE_REGISTRATION_MAX_ATTEMPTS)
val SLEEP_TIME_SECS = 5 val SLEEP_TIME_SECS = 5

View file

@ -21,11 +21,18 @@ import java.io.{File, IOException}
import java.nio.file.Files import java.nio.file.Files
import java.util.UUID import java.util.UUID
import scala.collection.mutable.HashMap
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import org.apache.spark.SparkConf import org.apache.spark.SparkConf
import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.executor.ExecutorExitCode
import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.{config, Logging}
import org.apache.spark.network.shuffle.ExecutorDiskUtils import org.apache.spark.network.shuffle.ExecutorDiskUtils
import org.apache.spark.storage.DiskBlockManager.MERGE_MANAGER_DIR import org.apache.spark.storage.DiskBlockManager.ATTEMPT_ID_KEY
import org.apache.spark.storage.DiskBlockManager.MERGE_DIR_KEY
import org.apache.spark.storage.DiskBlockManager.MERGE_DIRECTORY
import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.{ShutdownHookManager, Utils}
/** /**
@ -57,6 +64,10 @@ private[spark] class DiskBlockManager(conf: SparkConf, var deleteFilesOnStop: Bo
// of subDirs(i) is protected by the lock of subDirs(i) // of subDirs(i) is protected by the lock of subDirs(i)
private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
// Get merge directory name, append attemptId if there is any
private val mergeDirName =
s"$MERGE_DIRECTORY${conf.get(config.APP_ATTEMPT_ID).map(id => s"_$id").getOrElse("")}"
// Create merge directories // Create merge directories
createLocalDirsForMergedShuffleBlocks() createLocalDirsForMergedShuffleBlocks()
@ -200,12 +211,12 @@ private[spark] class DiskBlockManager(conf: SparkConf, var deleteFilesOnStop: Bo
// Will create the merge_manager directory only if it doesn't exist under the local dir. // Will create the merge_manager directory only if it doesn't exist under the local dir.
Utils.getConfiguredLocalDirs(conf).foreach { rootDir => Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
try { try {
val mergeDir = new File(rootDir, MERGE_MANAGER_DIR) val mergeDir = new File(rootDir, mergeDirName)
if (!mergeDir.exists()) { if (!mergeDir.exists()) {
// This executor does not find merge_manager directory, it will try to create // This executor does not find merge_manager directory, it will try to create
// the merge_manager directory and the sub directories. // the merge_manager directory and the sub directories.
logDebug(s"Try to create $mergeDir and its sub dirs since the " + logDebug(s"Try to create $mergeDir and its sub dirs since the " +
s"$MERGE_MANAGER_DIR dir does not exist") s"$mergeDirName dir does not exist")
for (dirNum <- 0 until subDirsPerLocalDir) { for (dirNum <- 0 until subDirsPerLocalDir) {
val subDir = new File(mergeDir, "%02x".format(dirNum)) val subDir = new File(mergeDir, "%02x".format(dirNum))
if (!subDir.exists()) { if (!subDir.exists()) {
@ -219,7 +230,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, var deleteFilesOnStop: Bo
} catch { } catch {
case e: IOException => case e: IOException =>
logError( logError(
s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e) s"Failed to create $mergeDirName dir in $rootDir. Ignoring this directory.", e)
} }
} }
} }
@ -264,6 +275,17 @@ private[spark] class DiskBlockManager(conf: SparkConf, var deleteFilesOnStop: Bo
} }
} }
def getMergeDirectoryAndAttemptIDJsonString(): String = {
val mergedMetaMap: HashMap[String, String] = new HashMap[String, String]()
mergedMetaMap.put(MERGE_DIR_KEY, mergeDirName)
conf.get(config.APP_ATTEMPT_ID).foreach(
attemptId => mergedMetaMap.put(ATTEMPT_ID_KEY, attemptId))
val mapper = new ObjectMapper()
mapper.registerModule(DefaultScalaModule)
val jsonString = mapper.writeValueAsString(mergedMetaMap)
jsonString
}
private def addShutdownHook(): AnyRef = { private def addShutdownHook(): AnyRef = {
logDebug("Adding shutdown hook") // force eager creation of logger logDebug("Adding shutdown hook") // force eager creation of logger
ShutdownHookManager.addShutdownHook(ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY + 1) { () => ShutdownHookManager.addShutdownHook(ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY + 1) { () =>
@ -303,5 +325,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, var deleteFilesOnStop: Bo
} }
private[spark] object DiskBlockManager { private[spark] object DiskBlockManager {
private[spark] val MERGE_MANAGER_DIR = "merge_manager" val MERGE_DIRECTORY = "merge_manager"
val MERGE_DIR_KEY = "mergeDir"
val ATTEMPT_ID_KEY = "attemptId"
} }

View file

@ -2592,32 +2592,13 @@ private[spark] object Utils extends Logging {
/** /**
* Push based shuffle can only be enabled when the application is submitted * Push based shuffle can only be enabled when the application is submitted
* to run in YARN mode, with external shuffle service enabled and * to run in YARN mode, with external shuffle service enabled
* spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
* TODO: Remove the requirement on spark.yarn.maxAttempts after SPARK-35546
* Support push based shuffle with multiple app attempts
*/ */
def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = { def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
conf.get(PUSH_BASED_SHUFFLE_ENABLED) && conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
(conf.get(IS_TESTING).getOrElse(false) || (conf.get(IS_TESTING).getOrElse(false) ||
(conf.get(SHUFFLE_SERVICE_ENABLED) && (conf.get(SHUFFLE_SERVICE_ENABLED) &&
conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn" && conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn"))
getYarnMaxAttempts(conf) == 1))
}
/**
* Returns the maximum number of attempts to register the AM in YARN mode.
* TODO: Remove this method after SPARK-35546 Support push based shuffle
* with multiple app attempts
*/
def getYarnMaxAttempts(conf: SparkConf): Int = {
val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
sparkMaxAttempts match {
case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
case None => yarnMaxAttempts
}
} }
/** /**

View file

@ -20,7 +20,10 @@ package org.apache.spark.storage
import java.io.{File, FileWriter} import java.io.{File, FileWriter}
import java.nio.file.{Files, Paths} import java.nio.file.{Files, Paths}
import java.nio.file.attribute.PosixFilePermissions import java.nio.file.attribute.PosixFilePermissions
import java.util.HashMap
import com.fasterxml.jackson.core.`type`.TypeReference
import com.fasterxml.jackson.databind.ObjectMapper
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
@ -91,11 +94,11 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
} }
test("should still create merge directories if one already exists under a local dir") { test("should still create merge directories if one already exists under a local dir") {
val mergeDir0 = new File(rootDir0, DiskBlockManager.MERGE_MANAGER_DIR) val mergeDir0 = new File(rootDir0, DiskBlockManager.MERGE_DIRECTORY)
if (!mergeDir0.exists()) { if (!mergeDir0.exists()) {
Files.createDirectories(mergeDir0.toPath) Files.createDirectories(mergeDir0.toPath)
} }
val mergeDir1 = new File(rootDir1, DiskBlockManager.MERGE_MANAGER_DIR) val mergeDir1 = new File(rootDir1, DiskBlockManager.MERGE_DIRECTORY)
if (mergeDir1.exists()) { if (mergeDir1.exists()) {
Utils.deleteRecursively(mergeDir1) Utils.deleteRecursively(mergeDir1)
} }
@ -104,7 +107,7 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
testConf.set(config.Tests.IS_TESTING, true) testConf.set(config.Tests.IS_TESTING, true)
diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true) diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
assert(Utils.getConfiguredLocalDirs(testConf).map( assert(Utils.getConfiguredLocalDirs(testConf).map(
rootDir => new File(rootDir, DiskBlockManager.MERGE_MANAGER_DIR)) rootDir => new File(rootDir, DiskBlockManager.MERGE_DIRECTORY))
.filter(mergeDir => mergeDir.exists()).length === 2) .filter(mergeDir => mergeDir.exists()).length === 2)
// mergeDir0 will be skipped as it already exists // mergeDir0 will be skipped as it already exists
assert(mergeDir0.list().length === 0) assert(mergeDir0.list().length === 0)
@ -124,6 +127,20 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
FileUtils.deleteQuietly(testDir) FileUtils.deleteQuietly(testDir)
} }
test("Encode merged directory name and attemptId in shuffleManager field") {
testConf.set(config.APP_ATTEMPT_ID, "1");
diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
val mergedShuffleMeta = diskBlockManager.getMergeDirectoryAndAttemptIDJsonString();
val mapper: ObjectMapper = new ObjectMapper
val typeRef: TypeReference[HashMap[String, String]] =
new TypeReference[HashMap[String, String]]() {}
val metaMap: HashMap[String, String] = mapper.readValue(mergedShuffleMeta, typeRef)
val mergeDir = metaMap.get(DiskBlockManager.MERGE_DIR_KEY)
assert(mergeDir.equals(DiskBlockManager.MERGE_DIRECTORY + "_1"))
val attemptId = metaMap.get(DiskBlockManager.ATTEMPT_ID_KEY)
assert(attemptId.equals("1"))
}
def writeToFile(file: File, numBytes: Int): Unit = { def writeToFile(file: File, numBytes: Int): Unit = {
val writer = new FileWriter(file, true) val writer = new FileWriter(file, true)
for (i <- 0 until numBytes) writer.write(i) for (i <- 0 until numBytes) writer.write(i)

View file

@ -1450,7 +1450,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
conf.set("spark.yarn.maxAttempts", "1") conf.set("spark.yarn.maxAttempts", "1")
assert(Utils.isPushBasedShuffleEnabled(conf) === true) assert(Utils.isPushBasedShuffleEnabled(conf) === true)
conf.set("spark.yarn.maxAttempts", "2") conf.set("spark.yarn.maxAttempts", "2")
assert(Utils.isPushBasedShuffleEnabled(conf) === false) assert(Utils.isPushBasedShuffleEnabled(conf) === true)
} }
} }