You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/07/07 09:17:07 UTC

[GitHub] [ratis] qian0817 opened a new pull request, #673: RATIS-1612. Support starting a server as a Listener.

qian0817 opened a new pull request, #673:
URL: https://github.com/apache/ratis/pull/673

   ## What changes were proposed in this pull request?
   
   Support starting a server as a Listener.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/RATIS-1612


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917821335


##########
ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java:
##########
@@ -118,6 +119,7 @@ static RaftPeer toRaftPeer(RaftPeerProto p) {
         .setClientAddress(p.getClientAddress())
         .setAdminAddress(p.getAdminAddress())
         .setPriority(p.getPriority())
+        .setStartupRole(p.hasStartupRole() ? p.getStartupRole() : RaftProtos.RaftPeerRole.FOLLOWER)

Review Comment:
   This is to ensure forward compatibility. Using the previous raft log will cause startupRole item to not exist.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917818527


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);
       if (follower == null) {
-        LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);
+        if (conf.getPeer(followerID, RaftPeerRole.LISTENER) == null) {

Review Comment:
   `senders.getSenders()` will return all peers including the listener.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r920646456


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -492,14 +492,18 @@ public void close() {
    * @param force Force to start a new {@link FollowerState} even if this server is already a follower.
    * @return if the term/votedFor should be updated to the new term
    */
-  private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
+  private synchronized boolean changeToFollower(
+      long newTerm,
+      boolean force,
+      boolean allowListener,
+      Object reason) {
     final RaftPeerRole old = role.getCurrentRole();
-    if (old == RaftPeerRole.LISTENER) {
+    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
+    if (old == RaftPeerRole.LISTENER && !allowListener) {
       throw new IllegalStateException("Unexpected role " + old);
     }
-    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
 
-    if (old != RaftPeerRole.FOLLOWER || force) {
+    if ((old != RaftPeerRole.FOLLOWER && old != RaftPeerRole.LISTENER) || force) {

Review Comment:
   fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r919568414


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
+import org.apache.ratis.proto.RaftProtos;

Review Comment:
   Change the import to
   ```
   import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
   ```
   It will make the code easier to read.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);
       if (follower == null) {
-        LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);
+        if (conf.getPeer(followerID, RaftPeerRole.LISTENER) == null) {
+          LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);

Review Comment:
   The existing code server.getId() was incorrect.  It should use followerID.
   ```
             LOG.error("{} the follower {} is not in the conf {}", this, followerID, conf);
   ```



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java:
##########
@@ -97,7 +97,7 @@ int size() {
 
   @Override
   public String toString() {
-    return peers.values().toString();
+    return "peers:" + peers.values() + "|listeners:" + listeners.toString();

Review Comment:
   It should be `listeners.values()`.



##########
ratis-server-api/src/main/java/org/apache/ratis/server/RaftServer.java:
##########
@@ -77,7 +80,14 @@ default RaftPeer getPeer() {
 
     /** @return the {@link RaftGroup} for this division. */
     default RaftGroup getGroup() {
-      return RaftGroup.valueOf(getMemberId().getGroupId(), getRaftConf().getAllPeers());
+      Collection<RaftPeer> allFollowerPeers =
+          getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER);
+      Collection<RaftPeer> allListenerPeers =
+          getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER);
+      Iterable<RaftPeer> peers = Iterables.concat(allFollowerPeers, allListenerPeers);
+      return RaftGroup.valueOf(
+          getMemberId().getGroupId(),
+          Lists.newArrayList(peers));

Review Comment:
   Change the `peers` parameter of `RaftGroup.valueOf(..)` to `Iterable<RaftPeer>`.  Then, we don't have to create a new array list.
   ```
       default RaftGroup getGroup() {
         final Collection<RaftPeer> followers = getRaftConf().getAllPeers(RaftPeerRole.FOLLOWER);
         final Collection<RaftPeer> listeners = getRaftConf().getAllPeers(RaftPeerRole.LISTENER);
         final Iterable<RaftPeer> peers = Iterables.concat(followers, listeners);
         return RaftGroup.valueOf(getMemberId().getGroupId(), peers);
       }
   ```
   



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -494,10 +494,10 @@ public void close() {
    */
   private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
     final RaftPeerRole old = role.getCurrentRole();
+    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
     if (old == RaftPeerRole.LISTENER) {
-      throw new IllegalStateException("Unexpected role " + old);
+      return metadataUpdated;
     }
-    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);

Review Comment:
   Let's add a new parameter `allowListener`.  We will allow listener in appendEntires and installSnapshot but not requestVote.
   ```
   @@ -494,9 +494,9 @@ class RaftServerImpl implements RaftServer.Division,
       * @param force Force to start a new {@link FollowerState} even if this server is already a follower.
       * @return if the term/votedFor should be updated to the new term
       */
   -  private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
   +  private synchronized boolean changeToFollower(long newTerm, boolean force, boolean allowListener, Object reason) {
        final RaftPeerRole old = role.getCurrentRole();
   -    if (old == RaftPeerRole.LISTENER) {
   +    if (old == RaftPeerRole.LISTENER && !allowListener) {
          throw new IllegalStateException("Unexpected role " + old);
        }
        final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
   ```



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);

Review Comment:
   Listener is a new feature.  The getPeer() method is for backward compatibility since it is in ratis-server-api. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on PR #673:
URL: https://github.com/apache/ratis/pull/673#issuecomment-1178792614

   @codings-dan @szetszwo Can you take a look at the new implementation?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo merged pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
szetszwo merged PR #673:
URL: https://github.com/apache/ratis/pull/673


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917829008


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);

Review Comment:
   To be honest the method name `conf.getPeer()` is confusing, I think most people will see this method and think it returns a collection of follower and listener. I think adding arguments would improve the readability of the code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r920346384


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -492,14 +492,18 @@ public void close() {
    * @param force Force to start a new {@link FollowerState} even if this server is already a follower.
    * @return if the term/votedFor should be updated to the new term
    */
-  private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
+  private synchronized boolean changeToFollower(
+      long newTerm,
+      boolean force,
+      boolean allowListener,
+      Object reason) {
     final RaftPeerRole old = role.getCurrentRole();
-    if (old == RaftPeerRole.LISTENER) {
+    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
+    if (old == RaftPeerRole.LISTENER && !allowListener) {
       throw new IllegalStateException("Unexpected role " + old);
     }
-    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
 
-    if (old != RaftPeerRole.FOLLOWER || force) {
+    if ((old != RaftPeerRole.FOLLOWER && old != RaftPeerRole.LISTENER) || force) {

Review Comment:
   This is better to be
   ```
   if ((old != RaftPeerRole.FOLLOWER || force) && old != RaftPeerRole.LISTENER) {
   ```
   Otherwise, `force == true` will change a Listener to a Follower.  Fortunately, when `force == true`, we always have `allowListener == false` in the current code.  Therefore, it won't cause any problems.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r919751924


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java:
##########
@@ -97,7 +97,7 @@ int size() {
 
   @Override
   public String toString() {
-    return peers.values().toString();
+    return "peers:" + peers.values() + "|listeners:" + listeners.toString();

Review Comment:
   fix



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -494,10 +494,10 @@ public void close() {
    */
   private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
     final RaftPeerRole old = role.getCurrentRole();
+    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
     if (old == RaftPeerRole.LISTENER) {
-      throw new IllegalStateException("Unexpected role " + old);
+      return metadataUpdated;
     }
-    final boolean metadataUpdated = state.updateCurrentTerm(newTerm);

Review Comment:
   fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] codings-dan commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
codings-dan commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917716420


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -133,10 +136,24 @@ public Builder setPriority(int priority) {
       return this;
     }
 
+    public Builder setStartupRole(RaftProtos.RaftPeerRole startupRole) {
+      if (startupRole != RaftProtos.RaftPeerRole.FOLLOWER
+          && startupRole != RaftProtos.RaftPeerRole.LISTENER) {
+        throw new IllegalArgumentException(
+            "At startup the role can only be set to FOLLOWER or LISTENER, the current value is " +
+                startupRole);
+      }
+      this.startupRole = startupRole;
+      return this;
+    }
+
     public RaftPeer build() {
+      if (startupRole == null) {

Review Comment:
   Please remove the if statement 



##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -75,6 +77,7 @@ public static class Builder {
     private String clientAddress;
     private String dataStreamAddress;
     private int priority;
+    private RaftProtos.RaftPeerRole startupRole;

Review Comment:
   set `startupRole` defalut value `RaftProtos.RaftPeerRole.FOLLOWER`



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);

Review Comment:
   There is no need to increase this variable, the default is to take FOLLOWER



##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java:
##########
@@ -75,6 +77,13 @@ public Collection<RaftPeer> getPeers() {
     return peers.values();
   }
 
+  public Collection<RaftPeer> getPeers(RaftProtos.RaftPeerRole role) {
+    return peers.values()
+        .stream()
+        .filter(peer -> peer.getStartupRole() == role)

Review Comment:
   I think we can't use `startupRole` as the difference between raftpeer, because the peer will change its identity



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);
       if (follower == null) {
-        LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);
+        if (conf.getPeer(followerID, RaftPeerRole.LISTENER) == null) {

Review Comment:
   Why do you need to add this judgment here? In the election process, it seems that listeners do not need to participate



##########
ratis-proto/src/main/proto/Raft.proto:
##########
@@ -28,6 +28,7 @@ message RaftPeerProto {
   string dataStreamAddress = 4; // address of the data stream server
   string clientAddress = 5; // address of the client RPC server
   string adminAddress = 6; // address of the admin RPC server
+  optional RaftPeerRole startupRole=7; // peer start up role

Review Comment:
   typo. `optional RaftPeerRole startupRole = 7`



##########
ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java:
##########
@@ -118,6 +119,7 @@ static RaftPeer toRaftPeer(RaftPeerProto p) {
         .setClientAddress(p.getClientAddress())
         .setAdminAddress(p.getAdminAddress())
         .setPriority(p.getPriority())
+        .setStartupRole(p.hasStartupRole() ? p.getStartupRole() : RaftProtos.RaftPeerRole.FOLLOWER)

Review Comment:
   Since the default value is set above, redundant judgment logic can be deleted here



##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -133,10 +136,24 @@ public Builder setPriority(int priority) {
       return this;
     }
 
+    public Builder setStartupRole(RaftProtos.RaftPeerRole startupRole) {
+      if (startupRole != RaftProtos.RaftPeerRole.FOLLOWER

Review Comment:
   I think `!=` can be replaced by `!...equals` here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on PR #673:
URL: https://github.com/apache/ratis/pull/673#issuecomment-1177807839

   > > @codings-dan I noticed that the Jira issue was assigned to you,can you take a look?
   > 
   > Sure , happy to help review this pull request. BTW, could you left a comment in the https://issues.apache.org/jira/browse/RATIS-1612 , and I will re-assign this jira to you, Thanks.
   
   done


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917822385


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -133,10 +136,24 @@ public Builder setPriority(int priority) {
       return this;
     }
 
+    public Builder setStartupRole(RaftProtos.RaftPeerRole startupRole) {
+      if (startupRole != RaftProtos.RaftPeerRole.FOLLOWER

Review Comment:
   Enume classes will have the same result when using `==` and `equals`. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917827211


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -75,6 +77,7 @@ public static class Builder {
     private String clientAddress;
     private String dataStreamAddress;
     private int priority;
+    private RaftProtos.RaftPeerRole startupRole;

Review Comment:
   fix.



##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java:
##########
@@ -75,6 +77,13 @@ public Collection<RaftPeer> getPeers() {
     return peers.values();
   }
 
+  public Collection<RaftPeer> getPeers(RaftProtos.RaftPeerRole role) {
+    return peers.values()
+        .stream()
+        .filter(peer -> peer.getStartupRole() == role)

Review Comment:
   fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] codings-dan commented on pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
codings-dan commented on PR #673:
URL: https://github.com/apache/ratis/pull/673#issuecomment-1177553537

   > @codings-dan I noticed that the Jira issue was assigned to you,can you take a look?
   
   Sure , happy to help review this pull request. BTW, could you left a comment in the https://issues.apache.org/jira/browse/RATIS-1612 , and I will re-assign this jira to you, Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on PR #673:
URL: https://github.com/apache/ratis/pull/673#issuecomment-1177300370

   @codings-dan I noticed that the Jira issue was assigned to you,can you take a look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on PR #673:
URL: https://github.com/apache/ratis/pull/673#issuecomment-1180177771

   Is this flaky test? i tried to run `./dev-support/checks/unit.sh -Pmisc-tests` on my machine but did not see the test fail.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] codings-dan commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
codings-dan commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r918886954


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -133,10 +136,24 @@ public Builder setPriority(int priority) {
       return this;
     }
 
+    public Builder setStartupRole(RaftProtos.RaftPeerRole startupRole) {
+      if (startupRole != RaftProtos.RaftPeerRole.FOLLOWER

Review Comment:
   ok, ignore my personal habits



##########
ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java:
##########
@@ -118,6 +119,7 @@ static RaftPeer toRaftPeer(RaftPeerProto p) {
         .setClientAddress(p.getClientAddress())
         .setAdminAddress(p.getAdminAddress())
         .setPriority(p.getPriority())
+        .setStartupRole(p.hasStartupRole() ? p.getStartupRole() : RaftProtos.RaftPeerRole.FOLLOWER)

Review Comment:
   Good Catch! I forget the forward compatibility



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);

Review Comment:
   But when designing this method parameter, we thought that the variable will only be specified when the Listener is obtained. Also, this kind of call is a lot in ratis, see below, so I think we should remove this change
   
   ![image](https://user-images.githubusercontent.com/26144703/178487125-a4b88819-9458-4095-aba7-820357ca78f2.png)
   



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);
       if (follower == null) {
-        LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);
+        if (conf.getPeer(followerID, RaftPeerRole.LISTENER) == null) {

Review Comment:
   Thanks for letting me know, I forgot about this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r917827035


##########
ratis-proto/src/main/proto/Raft.proto:
##########
@@ -28,6 +28,7 @@ message RaftPeerProto {
   string dataStreamAddress = 4; // address of the data stream server
   string clientAddress = 5; // address of the client RPC server
   string adminAddress = 6; // address of the admin RPC server
+  optional RaftPeerRole startupRole=7; // peer start up role

Review Comment:
   fix.



##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -133,10 +136,24 @@ public Builder setPriority(int priority) {
       return this;
     }
 
+    public Builder setStartupRole(RaftProtos.RaftPeerRole startupRole) {
+      if (startupRole != RaftProtos.RaftPeerRole.FOLLOWER
+          && startupRole != RaftProtos.RaftPeerRole.LISTENER) {
+        throw new IllegalArgumentException(
+            "At startup the role can only be set to FOLLOWER or LISTENER, the current value is " +
+                startupRole);
+      }
+      this.startupRole = startupRole;
+      return this;
+    }
+
     public RaftPeer build() {
+      if (startupRole == null) {

Review Comment:
   fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r919751700


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java:
##########
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
+import org.apache.ratis.proto.RaftProtos;

Review Comment:
   fix



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -951,9 +951,11 @@ private void yieldLeaderToHigherPriorityPeer() {
     for (LogAppender logAppender : senders.getSenders()) {
       final FollowerInfo followerInfo = logAppender.getFollower();
       final RaftPeerId followerID = followerInfo.getPeer().getId();
-      final RaftPeer follower = conf.getPeer(followerID);
+      final RaftPeer follower = conf.getPeer(followerID, RaftPeerRole.FOLLOWER);
       if (follower == null) {
-        LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);
+        if (conf.getPeer(followerID, RaftPeerRole.LISTENER) == null) {
+          LOG.error("{} the follower {} is not in the conf {}", this, server.getId(), conf);

Review Comment:
   fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] qian0817 commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
qian0817 commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r919752323


##########
ratis-server-api/src/main/java/org/apache/ratis/server/RaftServer.java:
##########
@@ -77,7 +80,14 @@ default RaftPeer getPeer() {
 
     /** @return the {@link RaftGroup} for this division. */
     default RaftGroup getGroup() {
-      return RaftGroup.valueOf(getMemberId().getGroupId(), getRaftConf().getAllPeers());
+      Collection<RaftPeer> allFollowerPeers =
+          getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER);
+      Collection<RaftPeer> allListenerPeers =
+          getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER);
+      Iterable<RaftPeer> peers = Iterables.concat(allFollowerPeers, allListenerPeers);
+      return RaftGroup.valueOf(
+          getMemberId().getGroupId(),
+          Lists.newArrayList(peers));

Review Comment:
   fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ratis] szetszwo commented on a diff in pull request #673: RATIS-1612. Support starting a server as a Listener.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #673:
URL: https://github.com/apache/ratis/pull/673#discussion_r916148743


##########
ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java:
##########
@@ -43,17 +43,28 @@ public static RaftGroup valueOf(RaftGroupId groupId, Collection<RaftPeer> peers)
     return new RaftGroup(groupId, peers);
   }
 
+  public static RaftGroup valueOf(RaftGroupId groupId, Collection<RaftPeer> peers,
+                                  Collection<RaftPeer> listeners) {
+    return new RaftGroup(groupId, peers, listeners);
+  }
+
   /** The group id */
   private final RaftGroupId groupId;
   /** The group of raft peers */
   private final Map<RaftPeerId, RaftPeer> peers;
+  private final Map<RaftPeerId, RaftPeer> listeners;

Review Comment:
   @qian0817 , thanks a lot for working on this.  Let's add a startupRole in RaftPeer instead of change RaftGroup.
   ```
   //RaftPeer
     private final RaftPeerRole startupRole;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org