This is an automated email from the ASF dual-hosted git repository.

zuston pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git


The following commit(s) were added to refs/heads/master by this push:
     new 1051d2635 [#1608][part-1] fix(spark): Only share the replacement 
servers for faulty servers in one stage (#1609)
1051d2635 is described below

commit 1051d26352b99a533e808b78835e26b338852331
Author: Junfan Zhang <[email protected]>
AuthorDate: Tue Apr 2 10:07:07 2024 +0800

    [#1608][part-1] fix(spark): Only share the replacement servers for faulty 
servers in one stage (#1609)
    
    ### What changes were proposed in this pull request?
    
    1. Lock the `shuffleHandle` to ensure the thread safe when reassigning 
partial server for tasks
    2. Only share the replacement servers for faulty servers in one stage 
rather than the whole app
    3. Simplify the reassignment logic, like the single one replacement server 
which will be supported in the future, so let's remove it currently.
    4. correct the `partitionIds` type from `string` to `int` in proto
    
    ### Why are the changes needed?
    
    Fix: #1608
    
    In current implementation of partition reassignment, it will share the same 
reassignment servers for the different stages, which will crash for app without 
registry.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    UTs
---
 .../apache/spark/shuffle/ShuffleHandleInfo.java    | 41 +++++++++++++--
 .../manager/RssShuffleManagerInterface.java        |  2 +-
 .../spark/shuffle/ShuffleHandleInfoTest.java       | 61 ++++++++++++++++++++++
 .../shuffle/manager/DummyRssShuffleManager.java    |  2 +-
 .../apache/spark/shuffle/RssShuffleManager.java    | 52 +++++++-----------
 .../apache/spark/shuffle/RssShuffleManager.java    | 53 +++++++------------
 .../spark/shuffle/writer/RssShuffleWriter.java     | 13 +++--
 .../RssReassignFaultyShuffleServerRequest.java     |  6 +--
 proto/src/main/proto/Rss.proto                     |  2 +-
 9 files changed, 148 insertions(+), 84 deletions(-)

diff --git 
a/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfo.java
 
b/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfo.java
index de999ed2c..e54145cbf 100644
--- 
a/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfo.java
+++ 
b/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfo.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.uniffle.common.RemoteStorageInfo;
 import org.apache.uniffle.common.ShuffleServerInfo;
+import org.apache.uniffle.common.util.JavaUtils;
 
 /**
  * Class for holding, 1. partition ID -> shuffle servers mapping. 2. remote 
storage info
@@ -41,7 +42,10 @@ public class ShuffleHandleInfo implements Serializable {
   private Map<Integer, List<ShuffleServerInfo>> partitionToServers;
 
   // partitionId -> replica -> failover servers
-  private Map<Integer, Map<Integer, List<ShuffleServerInfo>>> 
failoverPartitionServers;
+  private Map<Integer, Map<Integer, ShuffleServerInfo>> 
failoverPartitionServers;
+  // todo: support mores replacement servers for one faulty server.
+  private Map<String, ShuffleServerInfo> faultyServerReplacements;
+
   // shuffle servers which is for store shuffle data
   private Set<ShuffleServerInfo> shuffleServersForData;
   // remoteStorage used for this job
@@ -62,16 +66,13 @@ public class ShuffleHandleInfo implements Serializable {
       this.shuffleServersForData.addAll(ssis);
     }
     this.remoteStorage = storageInfo;
+    this.faultyServerReplacements = JavaUtils.newConcurrentMap();
   }
 
   public Map<Integer, List<ShuffleServerInfo>> getPartitionToServers() {
     return partitionToServers;
   }
 
-  public Map<Integer, Map<Integer, List<ShuffleServerInfo>>> 
getFailoverPartitionServers() {
-    return failoverPartitionServers;
-  }
-
   public Set<ShuffleServerInfo> getShuffleServersForData() {
     return shuffleServersForData;
   }
@@ -83,4 +84,34 @@ public class ShuffleHandleInfo implements Serializable {
   public int getShuffleId() {
     return shuffleId;
   }
+
+  public boolean isExistingFaultyServer(String serverId) {
+    return faultyServerReplacements.containsKey(serverId);
+  }
+
+  public ShuffleServerInfo useExistingReassignmentForMultiPartitions(
+      Set<Integer> partitionIds, String faultyServerId) {
+    return createNewReassignmentForMultiPartitions(partitionIds, 
faultyServerId, null);
+  }
+
+  public ShuffleServerInfo createNewReassignmentForMultiPartitions(
+      Set<Integer> partitionIds, String faultyServerId, ShuffleServerInfo 
replacement) {
+    if (replacement != null) {
+      faultyServerReplacements.put(faultyServerId, replacement);
+    }
+
+    replacement = faultyServerReplacements.get(faultyServerId);
+    for (Integer partitionId : partitionIds) {
+      List<ShuffleServerInfo> replicaServers = 
partitionToServers.get(partitionId);
+      for (int i = 0; i < replicaServers.size(); i++) {
+        if (replicaServers.get(i).getId().equals(faultyServerId)) {
+          Map<Integer, ShuffleServerInfo> replicaReplacements =
+              failoverPartitionServers.computeIfAbsent(
+                  partitionId, k -> JavaUtils.newConcurrentMap());
+          replicaReplacements.put(i, replacement);
+        }
+      }
+    }
+    return replacement;
+  }
 }
diff --git 
a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java
 
b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java
index 3160cc6dc..51d191a68 100644
--- 
a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java
+++ 
b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java
@@ -79,5 +79,5 @@ public interface RssShuffleManagerInterface {
       int stageId, int stageAttemptNumber, int shuffleId, int numMaps);
 
   ShuffleServerInfo reassignFaultyShuffleServerForTasks(
-      int shuffleId, Set<String> partitionIds, String faultyShuffleServerId);
+      int shuffleId, Set<Integer> partitionIds, String faultyShuffleServerId);
 }
diff --git 
a/client-spark/common/src/test/java/org/apache/spark/shuffle/ShuffleHandleInfoTest.java
 
b/client-spark/common/src/test/java/org/apache/spark/shuffle/ShuffleHandleInfoTest.java
new file mode 100644
index 000000000..bb1b1ca77
--- /dev/null
+++ 
b/client-spark/common/src/test/java/org/apache/spark/shuffle/ShuffleHandleInfoTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.jupiter.api.Test;
+
+import org.apache.uniffle.common.RemoteStorageInfo;
+import org.apache.uniffle.common.ShuffleServerInfo;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ShuffleHandleInfoTest {
+
+  private ShuffleServerInfo createFakeServerInfo(String host) {
+    return new ShuffleServerInfo(host, 1);
+  }
+
+  @Test
+  public void testReassignment() {
+    Map<Integer, List<ShuffleServerInfo>> partitionToServers = new HashMap<>();
+    partitionToServers.put(1, Arrays.asList(createFakeServerInfo("a"), 
createFakeServerInfo("b")));
+    partitionToServers.put(2, Arrays.asList(createFakeServerInfo("c")));
+
+    ShuffleHandleInfo handleInfo =
+        new ShuffleHandleInfo(1, partitionToServers, new 
RemoteStorageInfo(""));
+
+    // case1
+    assertFalse(handleInfo.isExistingFaultyServer("a"));
+    Set<Integer> partitions = new HashSet<>();
+    partitions.add(1);
+    ShuffleServerInfo newServer = createFakeServerInfo("d");
+    handleInfo.createNewReassignmentForMultiPartitions(partitions, "a", 
createFakeServerInfo("d"));
+    assertTrue(handleInfo.isExistingFaultyServer("a"));
+
+    assertEquals(newServer, 
handleInfo.useExistingReassignmentForMultiPartitions(partitions, "a"));
+  }
+}
diff --git 
a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java
 
b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java
index 841e13fa3..37d2b8eff 100644
--- 
a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java
+++ 
b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java
@@ -70,7 +70,7 @@ public class DummyRssShuffleManager implements 
RssShuffleManagerInterface {
 
   @Override
   public ShuffleServerInfo reassignFaultyShuffleServerForTasks(
-      int shuffleId, Set<String> partitionIds, String faultyShuffleServerId) {
+      int shuffleId, Set<Integer> partitionIds, String faultyShuffleServerId) {
     return mock(ShuffleServerInfo.class);
   }
 }
diff --git 
a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
 
b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
index 4600bcc6e..78bcc2c17 100644
--- 
a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
+++ 
b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
@@ -32,8 +32,6 @@ import scala.collection.Iterator;
 import scala.collection.Seq;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.spark.ShuffleDependency;
@@ -140,8 +138,6 @@ public class RssShuffleManager extends 
RssShuffleManagerBase {
    */
   private Map<String, Boolean> serverAssignedInfos = 
JavaUtils.newConcurrentMap();
 
-  private Map<String, ShuffleServerInfo> reassignedFaultyServers = 
JavaUtils.newConcurrentMap();
-
   public RssShuffleManager(SparkConf sparkConf, boolean isDriver) {
     if (sparkConf.getBoolean("spark.sql.adaptive.enabled", false)) {
       throw new IllegalArgumentException(
@@ -895,39 +891,27 @@ public class RssShuffleManager extends 
RssShuffleManagerBase {
     }
   }
 
+  // this is only valid on driver side that exposed to being invoked by grpc 
server
   @Override
   public ShuffleServerInfo reassignFaultyShuffleServerForTasks(
-      int shuffleId, Set<String> partitionIds, String faultyShuffleServerId) {
-    ShuffleServerInfo newShuffleServerInfo =
-        reassignedFaultyServers.computeIfAbsent(
-            faultyShuffleServerId,
-            id -> {
-              ShuffleServerInfo newAssignedServer = 
assignShuffleServer(shuffleId, id);
-              ShuffleHandleInfo shuffleHandleInfo = 
shuffleIdToShuffleHandleInfo.get(shuffleId);
-              for (String partitionId : partitionIds) {
-                Integer partitionIdInteger = Integer.valueOf(partitionId);
-                List<ShuffleServerInfo> shuffleServerInfoList =
-                    
shuffleHandleInfo.getPartitionToServers().get(partitionIdInteger);
-                for (int i = 0; i < shuffleServerInfoList.size(); i++) {
-                  if 
(shuffleServerInfoList.get(i).getId().equals(faultyShuffleServerId)) {
-                    shuffleHandleInfo
-                        .getFailoverPartitionServers()
-                        .computeIfAbsent(partitionIdInteger, k -> 
Maps.newHashMap());
-                    shuffleHandleInfo
-                        .getFailoverPartitionServers()
-                        .get(partitionIdInteger)
-                        .computeIfAbsent(i, j -> Lists.newArrayList())
-                        .add(newAssignedServer);
-                  }
-                }
-              }
-              return newAssignedServer;
-            });
-    return newShuffleServerInfo;
-  }
+      int shuffleId, Set<Integer> partitionIds, String faultyShuffleServerId) {
+    ShuffleHandleInfo handleInfo = shuffleIdToShuffleHandleInfo.get(shuffleId);
+    synchronized (handleInfo) {
+      // find out whether this server has been marked faulty in this shuffle
+      // if it has been reassigned, directly return the replacement server.
+      if (handleInfo.isExistingFaultyServer(faultyShuffleServerId)) {
+        return handleInfo.useExistingReassignmentForMultiPartitions(
+            partitionIds, faultyShuffleServerId);
+      }
 
-  public Map<String, ShuffleServerInfo> getReassignedFaultyServers() {
-    return reassignedFaultyServers;
+      // get the newer server to replace faulty server.
+      ShuffleServerInfo newAssignedServer = assignShuffleServer(shuffleId, 
faultyShuffleServerId);
+      if (newAssignedServer != null) {
+        handleInfo.createNewReassignmentForMultiPartitions(
+            partitionIds, faultyShuffleServerId, newAssignedServer);
+      }
+      return newAssignedServer;
+    }
   }
 
   private ShuffleServerInfo assignShuffleServer(int shuffleId, String 
faultyShuffleServerId) {
diff --git 
a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
 
b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
index 719aa80e5..0b4faef82 100644
--- 
a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
+++ 
b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java
@@ -35,8 +35,6 @@ import scala.collection.Iterator;
 import scala.collection.Seq;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.hadoop.conf.Configuration;
@@ -149,8 +147,6 @@ public class RssShuffleManager extends 
RssShuffleManagerBase {
    */
   private Map<String, Boolean> serverAssignedInfos;
 
-  private Map<String, ShuffleServerInfo> reassignedFaultyServers;
-
   public RssShuffleManager(SparkConf conf, boolean isDriver) {
     this.sparkConf = conf;
     boolean supportsRelocation =
@@ -282,7 +278,6 @@ public class RssShuffleManager extends 
RssShuffleManagerBase {
     this.shuffleIdToShuffleHandleInfo = JavaUtils.newConcurrentMap();
     this.failuresShuffleServerIds = Sets.newHashSet();
     this.serverAssignedInfos = JavaUtils.newConcurrentMap();
-    this.reassignedFaultyServers = JavaUtils.newConcurrentMap();
   }
 
   public CompletableFuture<Long> sendData(AddBlockEvent event) {
@@ -1187,39 +1182,27 @@ public class RssShuffleManager extends 
RssShuffleManagerBase {
     }
   }
 
+  // this is only valid on driver side that exposed to being invoked by grpc 
server
   @Override
   public ShuffleServerInfo reassignFaultyShuffleServerForTasks(
-      int shuffleId, Set<String> partitionIds, String faultyShuffleServerId) {
-    ShuffleServerInfo newShuffleServerInfo =
-        reassignedFaultyServers.computeIfAbsent(
-            faultyShuffleServerId,
-            id -> {
-              ShuffleServerInfo newAssignedServer = 
assignShuffleServer(shuffleId, id);
-              ShuffleHandleInfo shuffleHandleInfo = 
shuffleIdToShuffleHandleInfo.get(shuffleId);
-              for (String partitionId : partitionIds) {
-                Integer partitionIdInteger = Integer.valueOf(partitionId);
-                List<ShuffleServerInfo> shuffleServerInfoList =
-                    
shuffleHandleInfo.getPartitionToServers().get(partitionIdInteger);
-                for (int i = 0; i < shuffleServerInfoList.size(); i++) {
-                  if 
(shuffleServerInfoList.get(i).getId().equals(faultyShuffleServerId)) {
-                    shuffleHandleInfo
-                        .getFailoverPartitionServers()
-                        .computeIfAbsent(partitionIdInteger, k -> 
Maps.newHashMap());
-                    shuffleHandleInfo
-                        .getFailoverPartitionServers()
-                        .get(partitionIdInteger)
-                        .computeIfAbsent(i, j -> Lists.newArrayList())
-                        .add(newAssignedServer);
-                  }
-                }
-              }
-              return newAssignedServer;
-            });
-    return newShuffleServerInfo;
-  }
+      int shuffleId, Set<Integer> partitionIds, String faultyShuffleServerId) {
+    ShuffleHandleInfo handleInfo = shuffleIdToShuffleHandleInfo.get(shuffleId);
+    synchronized (handleInfo) {
+      // find out whether this server has been marked faulty in this shuffle
+      // if it has been reassigned, directly return the replacement server.
+      if (handleInfo.isExistingFaultyServer(faultyShuffleServerId)) {
+        return handleInfo.useExistingReassignmentForMultiPartitions(
+            partitionIds, faultyShuffleServerId);
+      }
 
-  public Map<String, ShuffleServerInfo> getReassignedFaultyServers() {
-    return reassignedFaultyServers;
+      // get the newer server to replace faulty server.
+      ShuffleServerInfo newAssignedServer = assignShuffleServer(shuffleId, 
faultyShuffleServerId);
+      if (newAssignedServer != null) {
+        handleInfo.createNewReassignmentForMultiPartitions(
+            partitionIds, faultyShuffleServerId, newAssignedServer);
+      }
+      return newAssignedServer;
+    }
   }
 
   private ShuffleServerInfo assignShuffleServer(int shuffleId, String 
faultyShuffleServerId) {
diff --git 
a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java
 
b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java
index c95bc17e0..635b3593a 100644
--- 
a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java
+++ 
b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -121,6 +122,8 @@ public class RssShuffleWriter<K, V, C> extends 
ShuffleWriter<K, V> {
 
   private final BlockingQueue<Object> finishEventQueue = new 
LinkedBlockingQueue<>();
 
+  private final Map<String, ShuffleServerInfo> faultyServers = new HashMap<>();
+
   // Only for tests
   @VisibleForTesting
   public RssShuffleWriter(
@@ -462,13 +465,12 @@ public class RssShuffleWriter<K, V, C> extends 
ShuffleWriter<K, V> {
     List<ShuffleBlockInfo> failedBlockInfoList = Lists.newArrayList();
     Map<ShuffleServerInfo, List<TrackingBlockStatus>> faultyServerToPartitions 
=
         failedBlockStatusSet.stream().collect(Collectors.groupingBy(d -> 
d.getShuffleServerInfo()));
-    Map<String, ShuffleServerInfo> faultyServers = 
shuffleManager.getReassignedFaultyServers();
     faultyServerToPartitions.entrySet().stream()
         .forEach(
             t -> {
-              Set<String> partitionIds =
+              Set<Integer> partitionIds =
                   t.getValue().stream()
-                      .map(x -> 
String.valueOf(x.getShuffleBlockInfo().getPartitionId()))
+                      .map(x -> x.getShuffleBlockInfo().getPartitionId())
                       .collect(Collectors.toSet());
               ShuffleServerInfo dynamicShuffleServer = 
faultyServers.get(t.getKey().getId());
               if (dynamicShuffleServer == null) {
@@ -519,7 +521,7 @@ public class RssShuffleWriter<K, V, C> extends 
ShuffleWriter<K, V> {
   }
 
   private ShuffleServerInfo reAssignFaultyShuffleServer(
-      Set<String> partitionIds, String faultyServerId) {
+      Set<Integer> partitionIds, String faultyServerId) {
     RssConf rssConf = RssSparkConfig.toRssConf(sparkConf);
     String driver = rssConf.getString("driver.host", "");
     int port = rssConf.get(RssClientConf.SHUFFLE_MANAGER_GRPC_PORT);
@@ -532,6 +534,9 @@ public class RssShuffleWriter<K, V, C> extends 
ShuffleWriter<K, V> {
         throw new RssException(
             "reassign server response with statusCode[" + 
response.getStatusCode() + "]");
       }
+      if (response.getShuffleServer() == null) {
+        throw new RssException("empty newer reassignment server!");
+      }
       return response.getShuffleServer();
     } catch (Exception e) {
       throw new RssException(
diff --git 
a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReassignFaultyShuffleServerRequest.java
 
b/internal-client/src/main/java/org/apache/uniffle/client/request/RssReassignFaultyShuffleServerRequest.java
index ac96a9ee3..c85666ff5 100644
--- 
a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReassignFaultyShuffleServerRequest.java
+++ 
b/internal-client/src/main/java/org/apache/uniffle/client/request/RssReassignFaultyShuffleServerRequest.java
@@ -24,11 +24,11 @@ import org.apache.uniffle.proto.RssProtos;
 public class RssReassignFaultyShuffleServerRequest {
 
   private int shuffleId;
-  private Set<String> partitionIds;
+  private Set<Integer> partitionIds;
   private String faultyShuffleServerId;
 
   public RssReassignFaultyShuffleServerRequest(
-      int shuffleId, Set<String> partitionIds, String faultyShuffleServerId) {
+      int shuffleId, Set<Integer> partitionIds, String faultyShuffleServerId) {
     this.shuffleId = shuffleId;
     this.partitionIds = partitionIds;
     this.faultyShuffleServerId = faultyShuffleServerId;
@@ -38,7 +38,7 @@ public class RssReassignFaultyShuffleServerRequest {
     return shuffleId;
   }
 
-  public Set<String> getPartitionIds() {
+  public Set<Integer> getPartitionIds() {
     return partitionIds;
   }
 
diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto
index ac9c76288..9ac38b736 100644
--- a/proto/src/main/proto/Rss.proto
+++ b/proto/src/main/proto/Rss.proto
@@ -590,7 +590,7 @@ message ReassignServersReponse{
 
 message RssReassignFaultyShuffleServerRequest{
   int32 shuffleId  = 1;
-  repeated string partitionIds = 2;
+  repeated int32 partitionIds = 2;
   string faultyShuffleServerId = 3;
 }
 

Reply via email to