Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Randomly distribute spooling directories across buckets #13982

Merged
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ public class FileSystemExchange
private final long exchangeSourceHandleTargetDataSizeInBytes;
private final ExecutorService executor;

private final Map<Integer, String> randomizedPrefixes = new ConcurrentHashMap<>();
private final Map<Integer, URI> outputDirectories = new ConcurrentHashMap<>();

@GuardedBy("this")
private final Set<Integer> allSinks = new HashSet<>();
Expand Down Expand Up @@ -270,12 +270,10 @@ private ListenableFuture<Multimap<Integer, FileStatus>> getCommittedPartitions(i

private URI getTaskOutputDirectory(int taskPartitionId)
{
URI baseDirectory = baseDirectories.get(taskPartitionId % baseDirectories.size());
String randomizedHexPrefix = randomizedPrefixes.computeIfAbsent(taskPartitionId, ignored -> generateRandomizedHexPrefix());

// Add a randomized prefix to evenly distribute data into different S3 shards
// Data output file path format: {randomizedHexPrefix}.{queryId}.{stageId}.{sinkPartitionId}/{attemptId}/{sourcePartitionId}_{splitId}.data
return baseDirectory.resolve(randomizedHexPrefix + "." + exchangeContext.getQueryId() + "." + exchangeContext.getExchangeId() + "." + taskPartitionId + PATH_SEPARATOR);
return outputDirectories.computeIfAbsent(taskPartitionId, ignored -> baseDirectories.get(ThreadLocalRandom.current().nextInt(baseDirectories.size()))
.resolve(generateRandomizedHexPrefix() + "." + exchangeContext.getQueryId() + "." + exchangeContext.getExchangeId() + "." + taskPartitionId + PATH_SEPARATOR));
}

@Override
Expand Down