This is an automated email from the ASF dual-hosted git repository.
zhengchenyu pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git
The following commit(s) were added to refs/heads/branch-0.9 by this push:
new 19fdc296c [MINOR] fix(client/netty): ShuffleServerGrpcNettyClient
missing to send shuffleId and partitionIds for requirePreAllocation request
(#2053)
19fdc296c is described below
commit 19fdc296c2be7d60c3840e1bfa57c7ba88727db6
Author: maobaolong <[email protected]>
AuthorDate: Thu Aug 15 21:42:43 2024 +0800
[MINOR] fix(client/netty): ShuffleServerGrpcNettyClient missing to send
shuffleId and partitionIds for requirePreAllocation request (#2053)
### What changes were proposed in this pull request?
Add partitionIds and shuffleId to `RequireBufferRequest`.
### Why are the changes needed?
Without this changes, server cannot check limitHugePartition.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Test on local, start a rss cluster with netty, specific a small huge
partition size, you can see NoBufferForHugePartitionException
(cherry picked from commit 77319985d8fe9f63fe230dbb440e99b82bc35b12)
---
.../uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java | 5 +++++
1 file changed, 5 insertions(+)
diff --git
a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java
b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java
index 0c6860ad1..1d6f7fdab 100644
---
a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java
+++
b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java
@@ -17,6 +17,7 @@
package org.apache.uniffle.client.impl.grpc;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@@ -113,11 +114,13 @@ public class ShuffleServerGrpcNettyClient extends
ShuffleServerGrpcClient {
int shuffleId = stb.getKey();
int size = 0;
int blockNum = 0;
+ List<Integer> partitionIds = new ArrayList<>();
for (Map.Entry<Integer, List<ShuffleBlockInfo>> ptb :
stb.getValue().entrySet()) {
for (ShuffleBlockInfo sbi : ptb.getValue()) {
size += sbi.getSize();
blockNum++;
}
+ partitionIds.add(ptb.getKey());
}
SendShuffleDataRequest sendShuffleDataRequest =
@@ -137,6 +140,8 @@ public class ShuffleServerGrpcNettyClient extends
ShuffleServerGrpcClient {
long requireId =
requirePreAllocation(
request.getAppId(),
+ shuffleId,
+ partitionIds,
allocateSize,
request.getRetryMax(),
request.getRetryIntervalMax());