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/03/24 08:37:23 UTC

[GitHub] [ratis] szetszwo commented on a change in pull request #627: RATIS-1558. Support Listener in RaftServerImpl

szetszwo commented on a change in pull request #627:
URL: https://github.com/apache/ratis/pull/627#discussion_r834037907



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
##########
@@ -112,7 +112,7 @@ boolean lostMajorityHeartbeatsRecently() {
   @Override
   public  void run() {
     final TimeDuration sleepDeviationThreshold = server.getSleepDeviationThreshold();
-    while (isRunning && server.getInfo().isFollower()) {
+    while (isRunning && (server.getInfo().isFollower() || server.getInfo().isListener())) {

Review comment:
       Let's add a shouldRun() method.
   ```
   @@ -109,10 +110,19 @@ class FollowerState extends Daemon {
        return true;
      }
    
   +  private boolean shouldRun() {
   +    final DivisionInfo info = server.getInfo();
   +    final boolean run = isRunning && (info.isFollower() || info.isListener());
   +    if (!run) {
   +      LOG.info("{}: Stopping now (isRunning? {}, role = {})", this, isRunning, info.getCurrentRole());
   +    }
   +    return run;
   +  }
   +
      @Override
      public  void run() {
        final TimeDuration sleepDeviationThreshold = server.getSleepDeviationThreshold();
   -    while (isRunning && server.getInfo().isFollower()) {
   +    while (shouldRun()) {
          final TimeDuration electionTimeout = server.getRandomElectionTimeout();
          try {
            final TimeDuration extraSleep = electionTimeout.sleep();
   ```

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -1338,7 +1354,7 @@ private CommitInfoProto updateCommitInfoCache() {
       }
       followerState = updateLastRpcTime(FollowerState.UpdateType.APPEND_START);
 
-      // Check that the append entries are not inconsistent. There are 3
+      // Check that the append ents are not inconsistent. There are 3

Review comment:
       Typo?

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -335,11 +338,19 @@ static boolean registerMBean(
   }
 
   /**
-   * The peer belongs to the current configuration, should start as a follower
+   * The peer belongs to the current configuration, should start as a follower or listener
    */
-  private void startAsFollower() {
-    setRole(RaftPeerRole.FOLLOWER, "startAsFollower");
-    role.startFollowerState(this, "startAsFollower");
+  private void startAsPeer(RaftPeerRole roles) {
+    Object reason = "";
+    if (roles.equals(RaftPeerRole.FOLLOWER)) {
+      reason = "startAsFollower";
+      setRole(RaftPeerRole.FOLLOWER, reason);
+    } else if (roles.equals(RaftPeerRole.LISTENER)) {

Review comment:
       Since RaftPeerRole is enum, use == .  Also, let's add an else-clause to throw an exception.
   ```
       if (newRole == RaftPeerRole.FOLLOWER) {
         reason = "startAsFollower";
         setRole(RaftPeerRole.FOLLOWER, reason);
       } else if (newRole == RaftPeerRole.LISTENER) {
         reason = "startAsListener";
         setRole(RaftPeerRole.LISTENER, reason);
       } else {
         throw new IllegalArgumentException("Unexpected role " + newRole);
       }
   ```
   

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -488,12 +499,17 @@ private synchronized boolean changeToFollower(long newTerm, boolean force, Objec
     final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
 
     if (old != RaftPeerRole.FOLLOWER || force) {
-      setRole(RaftPeerRole.FOLLOWER, reason);
+      if (old != RaftPeerRole.LISTENER) {

Review comment:
       It is illegal to have old == LISTENER.  Let's throw an exception in the beginning.  Then we don't have to change other code in this method.
   ```
   @@ -485,6 +498,9 @@ class RaftServerImpl implements RaftServer.Division,
       */
      private synchronized boolean changeToFollower(long newTerm, boolean force, Object reason) {
        final RaftPeerRole old = role.getCurrentRole();
   +    if (old == RaftPeerRole.LISTENER) {
   +      throw new IllegalStateException("Unexpected role " + old);
   +    }
        final boolean metadataUpdated = state.updateCurrentTerm(newTerm);
   ```

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
##########
@@ -122,14 +122,14 @@ public  void run() {
           continue;
         }
 
-        final boolean isFollower = server.getInfo().isFollower();
-        if (!isRunning || !isFollower) {
-          LOG.info("{}: Stopping now (isRunning? {}, isFollower? {})", this, isRunning, isFollower);
+        final boolean isPeer = (server.getInfo().isFollower() || server.getInfo().isListener());
+        if (!isRunning || !isPeer) {
+          LOG.info("{}: Stopping now (isRunning? {}, isPeer? {})", this, isRunning, isPeer);

Review comment:
       Use shouldRun() from above.




-- 
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