mridulm commented on code in PR #3248:
URL: https://github.com/apache/celeborn/pull/3248#discussion_r2220239087
##########
client/src/test/java/org/apache/celeborn/client/write/DataPushQueueSuiteJ.java:
##########
@@ -124,7 +124,7 @@ protected void pushData(PushTask task) throws IOException {
for (int i = 0; i < numPartitions; i++) {
byte[] b = intToBytes(workerData.get(i % numWorker).get(i / numWorker));
int batchId = pushState.nextBatchId();
- pushState.addBatch(batchId, reducePartitionMap.get(i).hostAndPushPort());
+ pushState.addBatch(batchId, 0,
reducePartitionMap.get(i).hostAndPushPort());
Review Comment:
nit: `0` -> `b.length` ?
##########
common/src/main/java/org/apache/celeborn/common/write/InFlightRequestTracker.java:
##########
@@ -45,9 +45,17 @@ public class InFlightRequestTracker {
private final AtomicInteger batchId = new AtomicInteger();
private final ConcurrentHashMap<String, Set<Integer>>
inflightBatchesPerAddress =
JavaUtils.newConcurrentHashMap();
+ private final ConcurrentHashMap<String, LongAdder>
inflightBytesSizePerAddress =
+ JavaUtils.newConcurrentHashMap();
+ private final ConcurrentHashMap<Integer, Integer> inflightBatchBytesSize =
+ JavaUtils.newConcurrentHashMap();
private final int maxInFlightReqsTotal;
+ private final boolean maxInFlightBytesSizeInFlightEnabled;
Review Comment:
nit:
```suggestion
private final boolean maxBytesSizeInFlightEnabled;
```
or perhaps `maxInFlightBytesSizeEnabled` ?
Similarly for others below ...
##########
client/src/main/scala/org/apache/celeborn/client/LifecycleManager.scala:
##########
@@ -1048,13 +1048,26 @@ class LifecycleManager(val appUniqueId: String, val
conf: CelebornConf) extends
}
context.reply(pbGetShuffleIdResponse)
case None =>
- val pbGetShuffleIdResponse = {
- logInfo(
- s"there is no finished map stage associated with appShuffleId
$appShuffleId")
-
PbGetShuffleIdResponse.newBuilder().setShuffleId(UNKNOWN_APP_SHUFFLE_ID).setSuccess(
- false).build()
+ // If no valid finished map stage is found, check if there are any
shuffles
+ shuffleIds.values.map(v => v._1).toSeq.reverse.headOption match {
+ case Some(celebornShuffleId) =>
+ val pbGetShuffleIdResponse = {
+ logInfo(
+ s"No finished map stage found, using latest shuffleId
$celebornShuffleId for appShuffleId $appShuffleId appShuffleIdentifier
$appShuffleIdentifier isWriter $isWriter")
+
PbGetShuffleIdResponse.newBuilder().setShuffleId(celebornShuffleId).setSuccess(
+ true).build()
+ }
+ context.reply(pbGetShuffleIdResponse)
+ case None =>
+ val pbGetShuffleIdResponse = {
+ logInfo(
+ s"there is no finished map stage associated with
appShuffleId $appShuffleId")
+ PbGetShuffleIdResponse.newBuilder().setShuffleId(
+ UNKNOWN_APP_SHUFFLE_ID).setSuccess(
+ false).build()
+ }
+ context.reply(pbGetShuffleIdResponse)
Review Comment:
Is this relevant to the current PR ?
##########
common/src/main/java/org/apache/celeborn/common/write/InFlightRequestTracker.java:
##########
@@ -90,6 +113,12 @@ public Set<Integer> getBatchIdSetByAddressPair(String
hostAndPort) {
hostAndPort, pair -> ConcurrentHashMap.newKeySet());
}
+ public long getBatchBytesSizeByAddressPair(String hostAndPort) {
+ return inflightBytesSizePerAddress
+ .computeIfAbsent(hostAndPort, pair -> new LongAdder())
+ .longValue();
Review Comment:
Do we want to insert it if missing ?
Why not simply:
`Optional.ofNullable(inflightBytesSizePerAddress.get(hostAndPort)).orElse(0L)`
##########
common/src/main/java/org/apache/celeborn/common/write/InFlightRequestTracker.java:
##########
@@ -57,24 +65,39 @@ public InFlightRequestTracker(CelebornConf conf, PushState
pushState) {
this.pushState = pushState;
this.pushStrategy = PushStrategy.getStrategy(conf);
this.maxInFlightReqsTotal = conf.clientPushMaxReqsInFlightTotal();
+ this.maxInFlightBytesSizeInFlightEnabled =
conf.clientPushMaxBytesSizeInFlightEnabled();
+ this.maxInFlightBytesSizeInFlightTotal =
conf.clientPushMaxBytesSizeInFlightTotal();
+ this.maxInFlightBytesSizeInFlightPerWorker =
conf.clientPushMaxBytesSizeInFlightPerWorker();
}
- public void addBatch(int batchId, String hostAndPushPort) {
+ public void addBatch(int batchId, int batchBytesSize, String
hostAndPushPort) {
Set<Integer> batchIdSetPerPair =
inflightBatchesPerAddress.computeIfAbsent(
hostAndPushPort, id -> ConcurrentHashMap.newKeySet());
batchIdSetPerPair.add(batchId);
+ LongAdder bytesSizePerPair =
+ inflightBytesSizePerAddress.computeIfAbsent(hostAndPushPort, id -> new
LongAdder());
+ bytesSizePerPair.add(batchBytesSize);
+ inflightBatchBytesSize.put(batchId, batchBytesSize);
totalInflightReqs.increment();
+ totalInflightBytes.add(batchBytesSize);
}
public void removeBatch(int batchId, String hostAndPushPort) {
Set<Integer> batchIdSet = inflightBatchesPerAddress.get(hostAndPushPort);
+ long batchBytesSize = inflightBatchBytesSize.getOrDefault(batchId, 0);
+ inflightBatchBytesSize.remove(batchId);
Review Comment:
```suggestion
long batchBytesSize =
Optional.ofNullable(inflightBatchBytesSize.remove(batchId)).orElse(0L);
```
##########
common/src/main/java/org/apache/celeborn/common/write/InFlightRequestTracker.java:
##########
@@ -110,6 +140,11 @@ public boolean limitMaxInFlight(String hostAndPushPort)
throws IOException {
&& batchIdSet.size() <= currentMaxReqsInFlight) {
break;
}
+ if (maxInFlightBytesSizeInFlightEnabled
+ && (totalInflightBytes.sum() <= maxInFlightBytesSizeInFlightTotal
+ && batchBytesSize <= maxInFlightBytesSizeInFlightPerWorker))
{
Review Comment:
Should this be '||' ?
```suggestion
&& (totalInflightBytes.sum() <=
maxInFlightBytesSizeInFlightTotal ||
batchBytesSize <= maxInFlightBytesSizeInFlightPerWorker)) {
```
--
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]