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

tanxinyu pushed a commit to branch dev/1.3
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/dev/1.3 by this push:
     new 884b8163c78 [remove datanode] Fix concurrent modification when 
removing fails and rollback #14333 (#14339)
884b8163c78 is described below

commit 884b8163c78ab7b59af3a97f1c9c61f925012d2e
Author: Xiangpeng Hu <[email protected]>
AuthorDate: Fri Dec 6 12:58:16 2024 +0800

    [remove datanode] Fix concurrent modification when removing fails and 
rollback #14333 (#14339)
    
    (cherry picked from commit 695d3be5ccdf37da78330dc28a109bd42f24b470)
---
 .../src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git 
a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java
 
b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java
index 77ca4ea8c47..64e27ec7102 100644
--- 
a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java
+++ 
b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensus.java
@@ -61,6 +61,7 @@ import 
org.apache.iotdb.consensus.iot.snapshot.IoTConsensusRateLimiter;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
 
+import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -492,8 +493,9 @@ public class IoTConsensus implements IConsensus {
       deleteLocalPeer(groupId);
       return;
     }
-    String previousPeerListStr = impl.getConfiguration().toString();
-    for (Peer peer : impl.getConfiguration()) {
+    ImmutableList<Peer> currentMembers = 
ImmutableList.copyOf(impl.getConfiguration());
+    String previousPeerListStr = currentMembers.toString();
+    for (Peer peer : currentMembers) {
       if (!correctPeers.contains(peer)) {
         if (!impl.removeSyncLogChannel(peer)) {
           logger.error(

Reply via email to