RexXiong commented on code in PR #2362:
URL:
https://github.com/apache/incubator-celeborn/pull/2362#discussion_r1523349473
##########
client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java:
##########
@@ -124,7 +125,8 @@ private static final class CelebornInputStreamImpl extends
CelebornInputStream {
private final CelebornConf conf;
private final TransportClientFactory clientFactory;
private final String shuffleKey;
- private PartitionLocation[] locations;
+ private ArrayList<PartitionLocation> locations;
+ private ArrayList<PbStreamHandler> streamHandlers;
Review Comment:
Requiring the two objects to have the same size does not feel like a good
idea. Should we consider using a pair so that even if some handler fails, you
can still use those success handlers.
##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1699,16 +1699,25 @@ public CelebornInputStream readPartition(
int startMapIndex,
int endMapIndex,
ExceptionMaker exceptionMaker,
+ ArrayList<PartitionLocation> locations,
+ ArrayList<PbStreamHandler> streamHandlers,
+ int[] mapAttempts,
MetricsCallback metricsCallback)
throws IOException {
if (partitionId == Utils$.MODULE$.UNKNOWN_APP_SHUFFLE_ID()) {
logger.warn("Shuffle data is empty for shuffle {}:
UNKNOWN_APP_SHUFFLE_ID.", shuffleId);
return CelebornInputStream.empty();
}
- ReduceFileGroups fileGroups = updateFileGroup(shuffleId, partitionId);
- if (fileGroups.partitionGroups.isEmpty()
- || !fileGroups.partitionGroups.containsKey(partitionId)) {
+ if (mapAttempts == null) {
+ ReduceFileGroups fileGroups = updateFileGroup(shuffleId, partitionId);
+ mapAttempts = fileGroups.mapAttempts;
Review Comment:
why not use locations as the condition to refresh fileGroups? It's more
clear then mapAttempts.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]