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

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


The following commit(s) were added to refs/heads/master by this push:
     new e07bd37  RATIS-1551. Support listener in setConfiguration. (#622)
e07bd37 is described below

commit e07bd375c277e51c61ad6c7a4a12d03c8325e76d
Author: Yaolong Liu <[email protected]>
AuthorDate: Fri Mar 18 10:47:03 2022 +0800

    RATIS-1551. Support listener in setConfiguration. (#622)
---
 .../org/apache/ratis/server/RaftConfiguration.java | 35 ++++++----
 .../apache/ratis/server/impl/LeaderStateImpl.java  | 77 +++++++++++++++-------
 .../ratis/server/impl/PeerConfiguration.java       | 49 ++++++++++----
 .../ratis/server/impl/RaftConfigurationImpl.java   | 68 +++++++------------
 .../org/apache/ratis/server/impl/ServerState.java  |  6 ++
 .../ratis/server/impl/TestRaftConfiguration.java   |  4 +-
 6 files changed, 144 insertions(+), 95 deletions(-)

diff --git 
a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java 
b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
index 2708558..bccb456 100644
--- 
a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
+++ 
b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.server;
 
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 
@@ -41,25 +42,31 @@ public interface RaftConfiguration {
    * @return the peer corresponding to the given id;
    *         or return null if the peer is not in this configuration.
    */
-  RaftPeer getPeer(RaftPeerId id);
+  RaftPeer getPeer(RaftPeerId id, RaftPeerRole... roles);
 
-  /**
-   * @return the listener corresponding to the given id;
-   *         or return null if the listener is not in this configuration.
-   */
-  RaftPeer getListener(RaftPeerId id);
+  /** The same as getAllPeers(RaftPeerRole.FOLLOWER). */
+  default Collection<RaftPeer> getAllPeers() {
+    return getAllPeers(RaftPeerRole.FOLLOWER);
+  }
+
+  /** @return all the peers of the given role in the current configuration and 
the previous configuration. */
+  Collection<RaftPeer> getAllPeers(RaftPeerRole role);
 
-  /** @return all the peers in the current configuration and the previous 
configuration. */
-  Collection<RaftPeer> getAllPeers();
+  /** The same as getCurrentPeers(RaftPeerRole.FOLLOWER). */
+  default Collection<RaftPeer> getCurrentPeers() {
+    return getCurrentPeers(RaftPeerRole.FOLLOWER);
+  }
 
-  /** @return all the listeners in the current configuration and the previous 
configuration. */
-  Collection<RaftPeer> getAllListeners();
+  /** @return all the peers of the given role in the current configuration. */
+  Collection<RaftPeer> getCurrentPeers(RaftPeerRole roles);
 
-  /** @return all the peers in the current configuration. */
-  Collection<RaftPeer> getCurrentPeers();
+  /** The same as getPreviousPeers(RaftPeerRole.FOLLOWER). */
+  default Collection<RaftPeer> getPreviousPeers() {
+    return getPreviousPeers(RaftPeerRole.FOLLOWER);
+  }
 
-  /** @return all the peers in the previous configuration. */
-  Collection<RaftPeer> getPreviousPeers();
+  /** @return all the peers of the given role in the previous configuration. */
+  Collection<RaftPeer> getPreviousPeers(RaftPeerRole roles);
 
   /** @return the index of the corresponding log entry for the current 
configuration. */
   long getLogEntryIndex();
diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
index cb685ff..5390905 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
@@ -22,6 +22,7 @@ import 
org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
 import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase;
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
 import org.apache.ratis.proto.RaftProtos.StartLeaderElectionReplyProto;
 import org.apache.ratis.proto.RaftProtos.StartLeaderElectionRequestProto;
@@ -291,7 +292,12 @@ class LeaderStateImpl implements LeaderState {
     placeHolderIndex = raftLog.getNextIndex();
 
     senders = new SenderList();
-    addSenders(others, placeHolderIndex, true);
+    addSenders(others, placeHolderIndex, true, RaftPeerRole.FOLLOWER);
+
+    final Collection<RaftPeer> listeners = 
conf.getAllPeers(RaftPeerRole.LISTENER);
+    if (!listeners.isEmpty()) {
+      addSenders(listeners, placeHolderIndex, false, RaftPeerRole.LISTENER);
+    }
     voterLists = divideFollowers(conf);
   }
 
@@ -362,22 +368,26 @@ class LeaderStateImpl implements LeaderState {
     Preconditions.assertTrue(running && !inStagingState());
 
     final List<RaftPeer> peersInNewConf = request.getPeersInNewConf();
+    final List<RaftPeer> listenersInNewConf = request.getListenersInNewConf();
     final Collection<RaftPeer> peersToBootStrap = 
server.getRaftConf().filterNotContainedInConf(peersInNewConf);
+    final Collection<RaftPeer> listenersToBootStrap= 
server.getRaftConf().filterNotContainedInConf(listenersInNewConf);
 
     // add the request to the pending queue
     final PendingRequest pending = pendingRequests.addConfRequest(request);
 
     ConfigurationStagingState configurationStagingState = new 
ConfigurationStagingState(
-        peersToBootStrap, new PeerConfiguration(peersInNewConf));
+        peersToBootStrap, listenersToBootStrap, new 
PeerConfiguration(peersInNewConf, listenersInNewConf));
     Collection<RaftPeer> newPeers = configurationStagingState.getNewPeers();
+    Collection<RaftPeer> newListeners = 
configurationStagingState.getNewListeners();
     // set the staging state
     this.stagingState = configurationStagingState;
 
-    if (newPeers.isEmpty()) {
+    if (newPeers.isEmpty() && newListeners.isEmpty()) {
       applyOldNewConf();
     } else {
       // update the LeaderState's sender list
-      addAndStartSenders(newPeers);
+      addAndStartSenders(newPeers, RaftPeerRole.FOLLOWER);
+      addAndStartSenders(newListeners, RaftPeerRole.LISTENER);
     }
     return pending;
   }
@@ -486,20 +496,24 @@ class LeaderStateImpl implements LeaderState {
   /**
    * Update sender list for setConfiguration request
    */
-  void addAndStartSenders(Collection<RaftPeer> newPeers) {
-    addSenders(newPeers, RaftLog.LEAST_VALID_LOG_INDEX, 
false).forEach(LogAppender::start);
+  void addAndStartSenders(Collection<RaftPeer> newPeers, RaftPeerRole role) {
+    if (!newPeers.isEmpty()) {
+      addSenders(newPeers, RaftLog.LEAST_VALID_LOG_INDEX, false, 
role).forEach(LogAppender::start);
+    }
   }
 
-  Collection<LogAppender> addSenders(Collection<RaftPeer> newPeers, long 
nextIndex, boolean attendVote) {
+  Collection<LogAppender> addSenders(Collection<RaftPeer> newPeers, long 
nextIndex, boolean attendVote,
+      RaftPeerRole role) {
     final Timestamp t = 
Timestamp.currentTime().addTimeMs(-server.getMaxTimeoutMs());
     final List<LogAppender> newAppenders = newPeers.stream()
         .map(peer -> {
           final FollowerInfo f = new FollowerInfoImpl(server.getMemberId(), 
peer, t, nextIndex, attendVote);
-          LogAppender logAppender = server.newLogAppender(this, f);
           peerIdFollowerInfoMap.put(peer.getId(), f);
-          raftServerMetrics.addFollower(peer.getId());
-          logAppenderMetrics.addFollowerGauges(peer.getId(), f::getNextIndex, 
f::getMatchIndex, f::getLastRpcTime);
-          return logAppender;
+          if (role == RaftPeerRole.FOLLOWER) {
+            raftServerMetrics.addFollower(peer.getId());
+            logAppenderMetrics.addFollowerGauges(peer.getId(), 
f::getNextIndex, f::getMatchIndex, f::getLastRpcTime);
+          }
+          return server.newLogAppender(this, f);
         }).collect(Collectors.toList());
     senders.addAll(newAppenders);
     return newAppenders;
@@ -513,11 +527,17 @@ class LeaderStateImpl implements LeaderState {
 
   @Override
   public void restart(LogAppender sender) {
-    final FollowerInfo follower = sender.getFollower();
-    LOG.info("{}: Restarting {} for {}", this, 
JavaUtils.getClassSimpleName(sender.getClass()), follower.getName());
+    final FollowerInfo info = sender.getFollower();
+    LOG.info("{}: Restarting {} for {}", this, 
JavaUtils.getClassSimpleName(sender.getClass()), info.getName());
     sender.stop();
     senders.removeAll(Collections.singleton(sender));
-    addAndStartSenders(Collections.singleton(follower.getPeer()));
+
+    final RaftPeer peer = info.getPeer();
+    if (server.getRaftConf().containsInConf(peer.getId())) {
+      addAndStartSenders(Collections.singleton(peer), RaftPeerRole.FOLLOWER);
+    } else if (server.getRaftConf().containsInConf(peer.getId(), 
RaftPeerRole.LISTENER)) {
+      addAndStartSenders(Collections.singleton(peer), RaftPeerRole.LISTENER);
+    }
   }
 
   /**
@@ -525,7 +545,7 @@ class LeaderStateImpl implements LeaderState {
    */
   private void updateSenders(RaftConfigurationImpl conf) {
     Preconditions.assertTrue(conf.isStable() && !inStagingState());
-    stopAndRemoveSenders(s -> !conf.containsInConf(s.getFollowerId()));
+    stopAndRemoveSenders(s -> !conf.containsInConf(s.getFollowerId(), 
RaftPeerRole.FOLLOWER, RaftPeerRole.LISTENER));
   }
 
   void submitStepDownEvent(StepDownReason reason) {
@@ -824,7 +844,7 @@ class LeaderStateImpl implements LeaderState {
     } else { // the (new) log entry has been committed
       pendingRequests.replySetConfiguration(server::newSuccessReply);
       // if the leader is not included in the current configuration, step down
-      if (!conf.containsInConf(server.getId())) {
+      if (!conf.containsInConf(server.getId(), RaftPeerRole.FOLLOWER, 
RaftPeerRole.LISTENER)) {
         LOG.info("{} is not included in the new configuration {}. Will 
shutdown server...", this, conf);
         try {
           // leave some time for all RPC senders to send out new conf entry
@@ -852,7 +872,7 @@ class LeaderStateImpl implements LeaderState {
         .setConf(conf)
         .setLogEntryIndex(raftLog.getNextIndex())
         .build();
-    // stop the LogAppender if the corresponding follower is no longer in the 
conf
+    // stop the LogAppender if the corresponding follower and listener is no 
longer in the conf
     updateSenders(newConf);
     long index = raftLog.append(server.getState().getCurrentTerm(), newConf);
     updateConfiguration(index, newConf);
@@ -1016,18 +1036,25 @@ class LeaderStateImpl implements LeaderState {
     return senders.stream().mapToLong(s -> 
s.getFollower().getNextIndex()).toArray();
   }
 
+  static Map<RaftPeerId, RaftPeer> newMap(Collection<RaftPeer> peers, String 
str) {
+    Objects.requireNonNull(peers, () -> str + " == null");
+    final Map<RaftPeerId, RaftPeer> map = new HashMap<>();
+    for(RaftPeer p : peers) {
+      map.put(p.getId(), p);
+    }
+    return Collections.unmodifiableMap(map);
+  }
+
   private class ConfigurationStagingState {
     private final String name = server.getMemberId() + "-" + 
JavaUtils.getClassSimpleName(getClass());
     private final Map<RaftPeerId, RaftPeer> newPeers;
+    private final Map<RaftPeerId, RaftPeer> newListeners;
     private final PeerConfiguration newConf;
 
-    ConfigurationStagingState(Collection<RaftPeer> newPeers,
+    ConfigurationStagingState(Collection<RaftPeer> newPeers, 
Collection<RaftPeer> newListeners,
         PeerConfiguration newConf) {
-      Map<RaftPeerId, RaftPeer> map = new HashMap<>();
-      for (RaftPeer peer : newPeers) {
-        map.put(peer.getId(), peer);
-      }
-      this.newPeers = Collections.unmodifiableMap(map);
+      this.newPeers = newMap(newPeers, "peer");
+      this.newListeners = newMap(newListeners, "listeners");
       this.newConf = newConf;
     }
 
@@ -1043,6 +1070,10 @@ class LeaderStateImpl implements LeaderState {
       return newPeers.values();
     }
 
+    Collection<RaftPeer> getNewListeners() {
+      return newListeners.values();
+    }
+
     boolean contains(RaftPeerId peerId) {
       return newPeers.containsKey(peerId);
     }
diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
index 530b20c..83effa8 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.server.impl;
 
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.util.Preconditions;
@@ -24,6 +25,7 @@ import org.apache.ratis.util.Preconditions;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -75,12 +77,18 @@ class PeerConfiguration {
         .orElseGet(Collections::emptyMap);
   }
 
-  Collection<RaftPeer> getPeers() {
-    return Collections.unmodifiableCollection(peers.values());
+  private Map<RaftPeerId, RaftPeer> getPeerMap(RaftPeerRole r) {
+    if (r == RaftPeerRole.FOLLOWER) {
+      return peers;
+    } else if (r == RaftPeerRole.LISTENER) {
+      return listeners;
+    } else {
+      throw new IllegalArgumentException("Unexpected RaftPeerRole " + r);
+    }
   }
 
-  Collection<RaftPeer> getListeners() {
-    return Collections.unmodifiableCollection(listeners.values());
+  Collection<RaftPeer> getPeers(RaftPeerRole role) {
+    return Collections.unmodifiableCollection(getPeerMap(role).values());
   }
 
   int size() {
@@ -92,20 +100,37 @@ class PeerConfiguration {
     return peers.values().toString();
   }
 
-  RaftPeer getPeer(RaftPeerId id) {
-    return peers.get(id);
+  RaftPeer getPeer(RaftPeerId id, RaftPeerRole... roles) {
+    if (roles == null || roles.length == 0) {
+      return peers.get(id);
+    }
+    for(RaftPeerRole r : roles) {
+      final RaftPeer peer = getPeerMap(r).get(id);
+      if (peer != null) {
+        return peer;
+      }
+    }
+    return null;
   }
 
-  RaftPeer getListener(RaftPeerId id) {
-    return listeners.get(id);
+  boolean contains(RaftPeerId id) {
+    return contains(id, RaftPeerRole.FOLLOWER);
   }
 
-  boolean contains(RaftPeerId id) {
-    return peers.containsKey(id);
+  boolean contains(RaftPeerId id, RaftPeerRole r) {
+    return getPeerMap(r).containsKey(id);
   }
 
-  boolean containsListener(RaftPeerId id) {
-    return listeners.containsKey(id);
+  RaftPeerRole contains(RaftPeerId id, EnumSet<RaftPeerRole> roles) {
+    if (roles == null || roles.isEmpty()) {
+      return peers.containsKey(id)? RaftPeerRole.FOLLOWER: null;
+    }
+    for(RaftPeerRole r : roles) {
+      if (getPeerMap(r).containsKey(id)) {
+        return r;
+      }
+    }
+    return null;
   }
 
   List<RaftPeer> getOtherPeers(RaftPeerId selfId) {
diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
index 4f6c122..ed661a9 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.server.impl;
 
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftConfiguration;
@@ -26,10 +27,9 @@ import org.apache.ratis.util.Preconditions;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Objects;
-import java.util.function.BiFunction;
-import java.util.function.Function;
 import java.util.stream.Collectors;
 
 /**
@@ -147,13 +147,16 @@ final class RaftConfigurationImpl implements 
RaftConfiguration {
     return oldConf == null;
   }
 
-  boolean containsInConf(RaftPeerId peerId) {
-    return conf.contains(peerId);
+  boolean containsInConf(RaftPeerId peerId, RaftPeerRole... roles) {
+    if (roles == null || roles.length == 0) {
+      return conf.contains(peerId);
+    } else if (roles.length == 1) {
+      return conf.contains(peerId, roles[0]);
+    } else {
+      return conf.contains(peerId, EnumSet.of(roles[0], roles)) != null;
+    }
   }
 
-  boolean containsListenerInConf(RaftPeerId peerId) {
-    return conf.containsListener(peerId);
-  }
 
   boolean isHighestPriority(RaftPeerId peerId) {
     RaftPeer target = getPeer(peerId);
@@ -173,10 +176,6 @@ final class RaftConfigurationImpl implements 
RaftConfiguration {
     return oldConf != null && oldConf.contains(peerId);
   }
 
-  boolean containsListenerInOldConf(RaftPeerId peerId) {
-    return oldConf != null && oldConf.containsListener(peerId);
-  }
-
   /**
    * @return true iff the given peer is contained in conf and,
    *         if old conf exists, is contained in old conf.
@@ -186,48 +185,25 @@ final class RaftConfigurationImpl implements 
RaftConfiguration {
         (oldConf == null || containsInOldConf(peerId));
   }
 
-  boolean containsListenerInBothConfs(RaftPeerId peerId) {
-    return containsListenerInConf(peerId) &&
-        (oldConf == null || containsListenerInOldConf(peerId));
-  }
-
-  @Override
-  public RaftPeer getPeer(RaftPeerId id) {
-    return get(id, (c, peerId) -> c.getPeer(id));
-  }
-
   @Override
-  public RaftPeer getListener(RaftPeerId id) {
-    return get(id, (c, peerId) -> c.getListener(id));
-  }
-
-  private RaftPeer get(RaftPeerId id, BiFunction<PeerConfiguration, 
RaftPeerId, RaftPeer> getMethod) {
+  public RaftPeer getPeer(RaftPeerId id, RaftPeerRole... roles) {
     if (id == null) {
       return null;
     }
-    final RaftPeer peer = getMethod.apply(conf, id);
+    final RaftPeer peer = conf.getPeer(id, roles);
     if (peer != null) {
       return peer;
     } else if (oldConf != null) {
-      return getMethod.apply(oldConf, id);
+      return oldConf.getPeer(id, roles);
     }
     return null;
   }
 
   @Override
-  public Collection<RaftPeer> getAllPeers() {
-    return getAll(PeerConfiguration::getPeers);
-  }
-
-  @Override
-  public Collection<RaftPeer> getAllListeners() {
-    return getAll(PeerConfiguration::getListeners);
-  }
-
-  private Collection<RaftPeer> getAll(Function<PeerConfiguration, 
Collection<RaftPeer>> getMethod) {
-    final Collection<RaftPeer> peers = new ArrayList<>(getMethod.apply(conf));
+  public Collection<RaftPeer> getAllPeers(RaftPeerRole role) {
+    final Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers(role));
     if (oldConf != null) {
-      getMethod.apply(oldConf).stream()
+      oldConf.getPeers(role).stream()
           .filter(p -> !peers.contains(p))
           .forEach(peers::add);
     }
@@ -285,17 +261,19 @@ final class RaftConfigurationImpl implements 
RaftConfiguration {
 
   /** @return the peers which are not contained in conf. */
   Collection<RaftPeer> filterNotContainedInConf(List<RaftPeer> peers) {
-    return peers.stream().filter(p -> 
!containsInConf(p.getId())).collect(Collectors.toList());
+    return peers.stream()
+        .filter(p -> !containsInConf(p.getId(), RaftPeerRole.FOLLOWER, 
RaftPeerRole.LISTENER))
+        .collect(Collectors.toList());
   }
 
   @Override
-  public Collection<RaftPeer> getPreviousPeers() {
-    return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
+  public Collection<RaftPeer> getPreviousPeers(RaftPeerRole role) {
+    return oldConf != null ? oldConf.getPeers(role) : Collections.emptyList();
   }
 
   @Override
-  public Collection<RaftPeer> getCurrentPeers() {
-    return conf.getPeers();
+  public Collection<RaftPeer> getCurrentPeers(RaftPeerRole role) {
+    return conf.getPeers(role);
   }
 
   @Override
diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
index 2558ad4..f05776a 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -18,6 +18,7 @@
 package org.apache.ratis.server.impl;
 
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.protocol.exceptions.StateMachineException;
 import org.apache.ratis.server.RaftConfiguration;
@@ -43,6 +44,7 @@ import java.io.IOException;
 import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -384,6 +386,10 @@ class ServerState implements Closeable {
   void setRaftConf(RaftConfiguration conf) {
     configurationManager.addConfiguration(conf);
     server.getServerRpc().addRaftPeers(conf.getAllPeers());
+    final Collection<RaftPeer> listeners = 
conf.getAllPeers(RaftPeerRole.LISTENER);
+    if (!listeners.isEmpty()) {
+      server.getServerRpc().addRaftPeers(listeners);
+    }
     LOG.info("{}: set configuration {}", getMemberId(), conf);
     LOG.trace("{}: {}", getMemberId(), configurationManager);
   }
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
index 23cae78..a547078 100644
--- 
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
@@ -18,6 +18,7 @@
 package org.apache.ratis.server.impl;
 
 import org.apache.ratis.BaseTest;
+import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.server.RaftConfiguration;
@@ -37,7 +38,8 @@ public class TestRaftConfiguration extends BaseTest {
     Integer node3 = 2;
     PeerConfiguration peerConfig = new 
PeerConfiguration(raftPeersWithPriority(node1, node2, node3));
     RaftConfiguration config = 
RaftConfigurationImpl.newBuilder().setConf(peerConfig).build();
-    RaftPeer[] allRaftPeers = peerConfig.getPeers().toArray(new 
RaftPeer[peerConfig.getPeers().size()]);
+    RaftPeer[] allRaftPeers = 
peerConfig.getPeers(RaftProtos.RaftPeerRole.FOLLOWER).toArray(new 
RaftPeer[peerConfig.getPeers(
+        RaftProtos.RaftPeerRole.FOLLOWER).size()]);
 
     // First member should not have highest priority
     Assert.assertFalse(RaftServerTestUtil.isHighestPriority(config,

Reply via email to