You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by ru...@apache.org on 2020/11/09 12:24:58 UTC

[incubator-ratis] branch master updated: RATIS-1139. Use RaftPeer.Bulider instead of the constructors. (#263)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 0e2ddf8  RATIS-1139. Use RaftPeer.Bulider instead of the constructors. (#263)
0e2ddf8 is described below

commit 0e2ddf80ae77882c9924b90939e736e8e31905f9
Author: Tsz-Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Mon Nov 9 20:24:51 2020 +0800

    RATIS-1139. Use RaftPeer.Bulider instead of the constructors. (#263)
    
    * RATIS-1139. Use RaftPeer.Bulider instead of the constructors.
    
    * Fix checkstyle.
---
 .../java/org/apache/ratis/protocol/RaftPeer.java   | 31 ++++++-----------
 .../ratis/examples/common/SubCommandBase.java      |  5 ++-
 .../ratis/examples/counter/CounterCommon.java      |  9 +++--
 .../apache/ratis/logservice/server/LogServer.java  |  4 +--
 .../ratis/logservice/util/LogServiceUtils.java     |  9 +++--
 .../logservice/util/MetaServiceProtoUtil.java      |  2 +-
 .../apache/ratis/server/impl/RaftServerImpl.java   |  2 +-
 .../java/org/apache/ratis/MiniRaftCluster.java     |  4 +--
 .../ratis/server/impl/GroupManagementBaseTest.java |  8 ++---
 .../server/impl/RaftReconfigurationBaseTest.java   |  3 +-
 .../ratis/server/impl/TestLogAppenderMetrics.java  |  4 +--
 .../ratis/datastream/TestDataStreamNetty.java      |  2 +-
 .../ratis/server/impl/TestPeerConfiguration.java   | 18 ++++++++--
 .../ratis/server/impl/TestRaftConfiguration.java   | 39 ----------------------
 .../org/apache/ratis/util/TestPeerProxyMap.java    |  2 +-
 15 files changed, 51 insertions(+), 91 deletions(-)

diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
index 6e452f3..e83b640 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
@@ -18,6 +18,7 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.proto.RaftProtos.RaftPeerProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.NetUtils;
 
@@ -33,7 +34,7 @@ import java.util.function.Supplier;
  *
  * The objects of this class are immutable.
  */
-public class RaftPeer {
+public final class RaftPeer {
   private static final RaftPeer[] EMPTY_ARRAY = {};
 
   /** @return an empty array. */
@@ -66,6 +67,14 @@ public class RaftPeer {
       return this;
     }
 
+    public Builder setId(String id) {
+      return setId(RaftPeerId.valueOf(id));
+    }
+
+    public Builder setId(ByteString id) {
+      return setId(RaftPeerId.valueOf(id));
+    }
+
     public Builder setAddress(String address) {
       this.address = address;
       return this;
@@ -110,26 +119,6 @@ public class RaftPeer {
 
   private final Supplier<RaftPeerProto> raftPeerProto;
 
-  /** Construct a peer with the given id and a null address. */
-  public RaftPeer(RaftPeerId id) {
-    this(id, (String)null, 0);
-  }
-
-  /** Construct a peer with the given id and address. */
-  public RaftPeer(RaftPeerId id, InetSocketAddress address) {
-    this(id, address == null ? null : NetUtils.address2String(address), 0);
-  }
-
-  /** Construct a peer with the given id and address. */
-  public RaftPeer(RaftPeerId id, String address) {
-    this(id, address, 0);
-  }
-
-  /** Construct a peer with the given id, address, priority. */
-  public RaftPeer(RaftPeerId id, String address, int priority) {
-    this(id, address, null, priority);
-  }
-
   private RaftPeer(RaftPeerId id, String address, String dataStreamAddress, int priority) {
     this.id = Objects.requireNonNull(id, "id == null");
     this.address = address;
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/common/SubCommandBase.java b/ratis-examples/src/main/java/org/apache/ratis/examples/common/SubCommandBase.java
index 428afa4..ccb466b 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/common/SubCommandBase.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/common/SubCommandBase.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -41,8 +41,7 @@ public abstract class SubCommandBase {
   public static RaftPeer[] parsePeers(String peers) {
     return Stream.of(peers.split(",")).map(address -> {
       String[] addressParts = address.split(":");
-      return new RaftPeer(RaftPeerId.valueOf(addressParts[0]),
-          addressParts[1] + ":" + addressParts[2]);
+      return RaftPeer.newBuilder().setId(addressParts[0]).setAddress(addressParts[1] + ":" + addressParts[2]).build();
     }).toArray(RaftPeer[]::new);
   }
 
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/CounterCommon.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/CounterCommon.java
index 56b77fe..4d0ae8b 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/CounterCommon.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/CounterCommon.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,7 +21,6 @@ package org.apache.ratis.examples.counter;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.protocol.RaftPeerId;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -34,9 +33,9 @@ public final class CounterCommon {
   public static final List<RaftPeer> PEERS = new ArrayList<>(3);
 
   static {
-    PEERS.add(new RaftPeer(RaftPeerId.getRaftPeerId("n1"), "127.0.0.1:6000"));
-    PEERS.add(new RaftPeer(RaftPeerId.getRaftPeerId("n2"), "127.0.0.1:6001"));
-    PEERS.add(new RaftPeer(RaftPeerId.getRaftPeerId("n3"), "127.0.0.1:6002"));
+    PEERS.add(RaftPeer.newBuilder().setId("n1").setAddress("127.0.0.1:6000").build());
+    PEERS.add(RaftPeer.newBuilder().setId("n2").setAddress("127.0.0.1:6001").build());
+    PEERS.add(RaftPeer.newBuilder().setId("n3").setAddress("127.0.0.1:6002").build());
   }
 
   private CounterCommon() {
diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogServer.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogServer.java
index f1d9cf9..1a4da83 100644
--- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogServer.java
+++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogServer.java
@@ -115,7 +115,7 @@ public class LogServer extends BaseServer {
             RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(new File(opts.getWorkingDir())));
         }
         String id = opts.getHost() +"_" +  opts.getPort();
-        RaftPeer peer = new RaftPeer(RaftPeerId.valueOf(id), addr);
+        final RaftPeer peer = RaftPeer.newBuilder().setId(id).setAddress(addr).build();
         final RaftGroupId logServerGroupId = RaftGroupId.valueOf(opts.getLogServerGroupId());
         RaftGroup all = RaftGroup.valueOf(logServerGroupId, peer);
         RaftGroup meta = RaftGroup.valueOf(RaftGroupId.valueOf(opts.getMetaGroupId()), peers);
@@ -146,7 +146,7 @@ public class LogServer extends BaseServer {
                 .setProperties(properties)
                 .build();
         metaClient.io().send(() -> MetaServiceProtoUtil.toPingRequestProto(peer).toByteString());
-        daemon = new Daemon(new HeartbeatSender(new RaftPeer(raftServer.getId())),
+        daemon = new Daemon(new HeartbeatSender(RaftPeer.newBuilder().setId(raftServer.getId()).build()),
                 "heartbeat-Sender"+raftServer.getId());
         daemon.start();
     }
diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/LogServiceUtils.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/LogServiceUtils.java
index 723a0e0..d2d1720 100644
--- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/LogServiceUtils.java
+++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/LogServiceUtils.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,13 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.ratis.logservice.util;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.ratis.logservice.api.LogName;
 import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.protocol.RaftPeerId;
 
 import java.net.DatagramSocket;
 import java.net.InetAddress;
@@ -34,14 +32,15 @@ public final class LogServiceUtils {
     }
 
     public static Set<RaftPeer> getPeersFromIds(String identity) {
+
         return Stream.of(identity.split(",")).map(elem ->
-                new RaftPeer(RaftPeerId.valueOf(elem), elem.replace('_', ':'))
+                RaftPeer.newBuilder().setId(elem).setAddress(elem.replace('_', ':')).build()
         ).collect(Collectors.toSet());
     }
 
     public static Set<RaftPeer> getPeersFromQuorum(String identity) {
         return Stream.of(identity.split(",")).map(elem ->
-                new RaftPeer(RaftPeerId.valueOf(elem.replace(':', '_')), elem)
+                RaftPeer.newBuilder().setId(elem.replace(':', '_')).setAddress(elem).build()
         ).collect(Collectors.toSet());
     }
 
diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/MetaServiceProtoUtil.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/MetaServiceProtoUtil.java
index bad1c8d..720cfc4 100644
--- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/MetaServiceProtoUtil.java
+++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/util/MetaServiceProtoUtil.java
@@ -50,7 +50,7 @@ public final class MetaServiceProtoUtil {
     }
 
     public static RaftPeer toRaftPeer(RaftPeerProto p) {
-        return new RaftPeer(RaftPeerId.valueOf(p.getId()), p.getAddress());
+        return RaftPeer.newBuilder().setId(p.getId()).setAddress(p.getAddress()).build();
     }
 
     public static RaftGroup toRaftGroup(RaftGroupProto proto) {
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 03f25aa..42b3eb1 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -97,7 +97,7 @@ public class RaftServerImpl implements RaftServerProtocol, RaftServerAsynchronou
   private final LifeCycle lifeCycle;
   private final ServerState state;
   private final Supplier<RaftPeer> peerSupplier = JavaUtils.memoize(() ->
-      new RaftPeer(getId(), getServerRpc().getInetSocketAddress()));
+      RaftPeer.newBuilder().setId(getId()).setAddress(getServerRpc().getInetSocketAddress()).build());
   private final RoleInfo role;
 
   private final RetryCache retryCache;
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 6c27ffe..07e166b 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -213,7 +213,7 @@ public abstract class MiniRaftCluster implements Closeable {
   public static RaftGroup initRaftGroup(Collection<String> ids) {
     final RaftPeer[] peers = ids.stream()
         .map(RaftPeerId::valueOf)
-        .map(id -> new RaftPeer(id, NetUtils.createLocalServerAddress()))
+        .map(id -> RaftPeer.newBuilder().setId(id).setAddress(NetUtils.createLocalServerAddress()).build())
         .toArray(RaftPeer[]::new);
     return RaftGroup.valueOf(RaftGroupId.randomId(), peers);
   }
@@ -391,7 +391,7 @@ public abstract class MiniRaftCluster implements Closeable {
   }
 
   public static RaftPeer toRaftPeer(RaftServerProxy s) {
-    return new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress());
+    return RaftPeer.newBuilder().setId(s.getId()).setAddress(s.getServerRpc().getInetSocketAddress()).build();
   }
 
   public PeerChanges addNewPeers(int number, boolean startNewPeer)
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
index 3c2de57..631c8d4 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
@@ -103,11 +103,9 @@ public abstract class GroupManagementBaseTest extends BaseTest {
     List<RaftPeer> peersWithPriority = new ArrayList<>();
     for (int i = 0; i < peers.size(); i++) {
       RaftPeer peer = peers.get(i);
-      if (i == suggestedLeaderIndex) {
-        peersWithPriority.add(new RaftPeer(peer.getId(), peer.getAddress(), 2));
-      } else {
-          peersWithPriority.add(new RaftPeer(peer.getId(), peer.getAddress(), 1));
-      }
+      final int priority = i == suggestedLeaderIndex? 2: 1;
+      peersWithPriority.add(
+          RaftPeer.newBuilder().setId(peer.getId()).setAddress(peer.getAddress()).setPriority(priority).build());
     }
 
     final RaftGroup newGroup = RaftGroup.valueOf(RaftGroupId.randomId(), peersWithPriority);
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index 3675137..e2f61a8 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -98,7 +98,8 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste
       List<RaftPeer> peersWithPriority = new ArrayList<>();
       for (int i = 0; i < peers.size(); i++) {
         RaftPeer peer = peers.get(i);
-        peersWithPriority.add(new RaftPeer(peer.getId(), peer.getAddress(), i));
+        peersWithPriority.add(
+            RaftPeer.newBuilder().setId(peer.getId()).setAddress(peer.getAddress()).setPriority(i).build());
       }
 
       final RaftGroup newGroup = RaftGroup.valueOf(RaftGroupId.randomId(), peersWithPriority);
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java
index 47a430b..74148f9 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -46,7 +46,7 @@ public class TestLogAppenderMetrics {
   public void setup() {
     RaftGroupId raftGroupId = RaftGroupId.randomId();
     raftPeerId = RaftPeerId.valueOf("TestId");
-    RaftPeer raftPeer = new RaftPeer(raftPeerId);
+    final RaftPeer raftPeer = RaftPeer.newBuilder().setId(raftPeerId).build();
     RaftGroupMemberId raftGroupMemberId = RaftGroupMemberId.valueOf(raftPeerId, raftGroupId);
     LogAppender logAppender = mock(LogAppender.class);
     followerInfo = new TestFollowerInfo(raftGroupMemberId, raftPeer, Timestamp.currentTime(), 100L, true, 1000);
diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamNetty.java
index 8b65ba5..27a19a5 100644
--- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamNetty.java
+++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamNetty.java
@@ -86,7 +86,7 @@ public class TestDataStreamNetty extends DataStreamBaseTest {
     RaftGroupId groupId = RaftGroupId.randomId();
     long callId = 100;
     long longIndex = 200;
-    RaftPeer suggestedLeader = new RaftPeer(RaftPeerId.valueOf("s" + leaderIndex));
+    final RaftPeer suggestedLeader = RaftPeer.newBuilder().setId("s" + leaderIndex).build();
     RaftClientReply expectedClientReply = new RaftClientReply(clientId, suggestedLeader.getId(),
         groupId, callId, true, null, null, longIndex, null);
 
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/PeerConfigurationTest.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java
similarity index 80%
rename from ratis-server/src/test/java/org/apache/ratis/server/impl/PeerConfigurationTest.java
rename to ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java
index e40ad71..e1adcfa 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/PeerConfigurationTest.java
+++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.server.impl;
 
+import org.apache.ratis.BaseTest;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.junit.Test;
@@ -28,7 +29,18 @@ import java.util.stream.Collectors;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class PeerConfigurationTest {
+public class TestPeerConfiguration extends BaseTest {
+  @Test
+  public void testPeerConfiguration() {
+    final RaftPeer[] peers = {
+        RaftPeer.newBuilder().setId("s0").build(),
+        RaftPeer.newBuilder().setId("s1").build(),
+        RaftPeer.newBuilder().setId("s0").build(),
+    };
+    testFailureCase("Duplicated peers", () -> {
+      new PeerConfiguration(Arrays.asList(peers));
+    }, IllegalArgumentException.class);
+  }
 
   @Test
   public void testOddNodesQuorum() {
@@ -58,7 +70,9 @@ public class PeerConfigurationTest {
   }
 
   private Collection<RaftPeer> raftPeers(String... voters) {
-    return Arrays.stream(voters).map(voter -> new RaftPeer(RaftPeerId.valueOf(voter))).collect(Collectors.toSet());
+    return Arrays.stream(voters)
+        .map(id -> RaftPeer.newBuilder().setId(id).build())
+        .collect(Collectors.toSet());
   }
 
   private Collection<RaftPeerId> raftPeerIds(String... voters) {
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java
deleted file mode 100644
index 0f5d771..0000000
--- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.ratis.server.impl;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.junit.Test;
-
-import java.util.Arrays;
-
-public class TestRaftConfiguration extends BaseTest {
-  @Test
-  public void testPeerConfiguration() {
-    final RaftPeer[] peers = {
-        new RaftPeer(RaftPeerId.valueOf("s0")),
-        new RaftPeer(RaftPeerId.valueOf("s1")),
-        new RaftPeer(RaftPeerId.valueOf("s0")),
-    };
-    testFailureCase("Duplicated peers", () -> {
-      new PeerConfiguration(Arrays.asList(peers));
-    }, IllegalArgumentException.class);
-  }
-}
\ No newline at end of file
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java
index e0e1c49..62ec354 100644
--- a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java
@@ -52,7 +52,7 @@ public class TestPeerProxyMap extends BaseTest {
   public void testCloseDeadLock() throws Exception {
     final PeerProxyMap<DummyProxy> map = new PeerProxyMap<>("test", DummyProxy::new);
     final RaftPeerId id = RaftPeerId.valueOf("s0");
-    final RaftPeer peer = new RaftPeer(id);
+    final RaftPeer peer = RaftPeer.newBuilder().setId(id).build();
     map.computeIfAbsent(peer);
 
     final DummyProxy proxy = map.getProxy(id);