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/02/11 13:06:40 UTC

[GitHub] [ratis] codings-dan opened a new pull request #598: RATIS RATIS-1301. Support listener in ratis

codings-dan opened a new pull request #598:
URL: https://github.com/apache/ratis/pull/598


   ## What changes were proposed in this pull request?
   Support listener in ratis
   
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/RATIS-1301
   
   ## How was this patch tested?
   
   UT
   


-- 
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 edited a comment on pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan edited a comment on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044146744


   @szetszwo It seems that the stepdown leader related work may have bug, the previous problem did not reproduce, but a new problem appeared. 


-- 
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 change in pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #598:
URL: https://github.com/apache/ratis/pull/598#discussion_r809711714



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -353,6 +356,12 @@ private void startInitializing() {
     // do not start FollowerState
   }
 
+  private void startAsListener() {
+    setRole(RaftPeerRole.LISTENER, "startAsListener");
+    //TODO(codings-dan): ListenerState
+    lifeCycle.transition(RUNNING);
+  }
+

Review comment:
       Since the we have changed the RaftConfiguration related code. We may change RaftServerImpl and the related class code in the next jira.




-- 
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 #598: RATIS-1301. Support listener in ratis

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


   @szetszwo PTAL again.
   After this MR, we may complete the work according to this progress:
   Change the code related to RaftServerImpl to support starting in the form of Listener;
   Add ListenerState related code and modify LeaderStateImpl related code to support passing journals to Listener
   Do you have any suggestions for future work? 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] codings-dan commented on pull request #598: RATIS-1301. Support listener in RaftConfiguration

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


   > @szetszwo PTAL again. After this MR, we may complete the work according to this progress: Change the code related to RaftServerImpl to support starting in the form of Listener; Add ListenerState related code and modify LeaderStateImpl related code to support passing journals to Listener Do you have any suggestions for future work? Thanks!
   
   @szetszwo What do you think about the follow-up code development in this way? Can you give some suggestions? Thank you!


-- 
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 change in pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #598:
URL: https://github.com/apache/ratis/pull/598#discussion_r809707517



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -145,6 +149,10 @@ boolean containsInConf(RaftPeerId peerId) {
     return conf.contains(peerId);
   }
 
+  boolean containsInListenerConf(RaftPeerId peerId) {

Review comment:
       Thanks for the guidance




-- 
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 pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044130236


   @codings-dan , actually, it is an old test but not the newly added test.  Let's see if we can reproduce the failure.


-- 
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 #598: RATIS-1301. Support listener in ratis

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


   This pr is based on https://github.com/apache/ratis/pull/409 . Current, it has implemented the code related to leader election. I will continue to add the code related to transport log on it. My idea is that we can temporarily let the leader send logs to all listeners, and then we can consider iterative optimization, using followers or other listeners to pass logs.
   
   For the log, I think the leader only needs to send all the committed logs to the listener, so that the log in the listner does not need to be truncate. In addition, I think the listener only needs to receive the log, and does not need to apply to the state machine.
   
   Can you give some suggestions for the above ideas, thank you @szetszwo 


-- 
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 pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1037982892


   @codings-dan , thanks for working on this.  The change is quite big.  Let's separate it into two or more JIRAs.
   
   The first one could be changing RaftConfiguration and the related classes to support Listener.  This probably is the most important change.  We should think the API and data structure change in details.  It seems to me that we should not change RaftGroup at all since RaftGroup should not care about the roles of the peers.


-- 
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 pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044124999


   @codings-dan , please run it 100 times and see if it would 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 change in pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #598:
URL: https://github.com/apache/ratis/pull/598#discussion_r809707272



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
##########
@@ -36,9 +36,18 @@
    */
   RaftPeer getPeer(RaftPeerId id);
 
+  /**

Review comment:
       I have add related java doc.




-- 
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 #598: RATIS-1301. Support listener in ratis

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


   @szetszwo I change the code related to RaftConfiguration and RaftSeverImpl based on the logic that does not affect the original code. I added some listener related code according to the follower. Can you help review it, thx! 


-- 
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 edited a comment on pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan edited a comment on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044146744


   @szetszwo It seems that the stepdown leader related work may have bug, the previous problem did not reproduce, but a new problem appeared.


-- 
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 #598: RATIS-1301. Support listener in ratis

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


   @szetszwo It seems that the ci test of github is not stable enough, I can run UT locally.
   
   ![image](https://user-images.githubusercontent.com/26144703/154644322-0d110ffb-1a14-4566-9a24-39e486f0ebc2.png)
   


-- 
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 #598: RATIS-1301. Support listener in ratis

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


   @szetszwo It seems that the stepdown leader related work may have bug, the previous problem did not reproduce, but a new problem appeared [https://github.com/apache/ratis/runs/5244723553?check_suite_focus=true#step:5:543](url)


-- 
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 pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044058182


   @codings-dan, The testLeaderStepDown just has failed in the build.  Please take a look.
   see [502](https://github.com/apache/ratis/runs/5243753207?check_suite_focus=true#step:5:502)
   ```
   Error:  Tests run: 3, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 2.995 s <<< FAILURE!
    - in org.apache.ratis.netty.TestPreAppendLeaderStepDownWithNetty
   Error:  testLeaderStepDown(org.apache.ratis.netty.TestPreAppendLeaderStepDownWithNetty)
     Time elapsed: 0.601 s  <<< FAILURE!
   java.lang.AssertionError
   ```


-- 
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 change in pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #598:
URL: https://github.com/apache/ratis/pull/598#discussion_r809120905



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
##########
@@ -36,9 +36,18 @@
    */
   RaftPeer getPeer(RaftPeerId id);
 
+  /**

Review comment:
       Let's update the javadoc of the class.
   ```
   +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
   @@ -25,9 +25,16 @@ import java.util.Collection;
    /**
     * A configuration is a subset of the members in a {@link org.apache.ratis.protocol.RaftGroup}.
     * The configuration of a cluster may change from time to time.
   + *
   + * In a configuration,
   + * - the peers are voting members such as LEADER, CANDIDATE and FOLLOWER;
   + * - the listeners are non-voting members.
   + *
     * This class captures the current configuration and the previous configuration of a cluster.
     *
     * The objects of this class are immutable.
   + *
   + * @see org.apache.ratis.proto.RaftProtos.RaftPeerRole
     */
    public interface RaftConfiguration {
   ```

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
##########
@@ -36,23 +36,41 @@
  */
 class PeerConfiguration {
   private final Map<RaftPeerId, RaftPeer> peers;
+  private final Map<RaftPeerId, RaftPeer> listeners;

Review comment:
       Let's add some javadoc:
   ```
     /**
      * Peers are voting members such as LEADER, CANDIDATE and FOLLOWER
      * @see org.apache.ratis.proto.RaftProtos.RaftPeerRole
      */
     private final Map<RaftPeerId, RaftPeer> peers;
     /**
      * Listeners are non-voting members.
      * @see org.apache.ratis.proto.RaftProtos.RaftPeerRole#LISTENER
      */
     private final Map<RaftPeerId, RaftPeer> listeners;
   ```
   

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
##########
@@ -36,23 +36,41 @@
  */
 class PeerConfiguration {
   private final Map<RaftPeerId, RaftPeer> peers;
+  private final Map<RaftPeerId, RaftPeer> listeners;
 
   PeerConfiguration(Iterable<RaftPeer> peers) {
+    this(peers, Collections.emptyList());
+  }
+
+  PeerConfiguration(Iterable<RaftPeer> peers, Iterable<RaftPeer> listeners) {

Review comment:
       We may add a utility method
   ```
     static Map<RaftPeerId, RaftPeer> newMap(Iterable<RaftPeer> peers, String name, Map<RaftPeerId, RaftPeer> existing) {
       Objects.requireNonNull(peers, () -> name + " == null");
       final Map<RaftPeerId, RaftPeer> map = new HashMap<>();
       for(RaftPeer p : peers) {
         if (existing.containsKey(p.getId())) {
           throw new IllegalArgumentException("Failed to initialize " + name
               + ": Found " + p.getId() + " in existing peers " + existing);
         }
         final RaftPeer previous = map.putIfAbsent(p.getId(), p);
         if (previous != null) {
           throw new IllegalArgumentException("Failed to initialize " + name
               + ": Found duplicated ids " + p.getId() + " in " + peers);
         }
       }
       return Collections.unmodifiableMap(map);
     }
   ```
   Then, the constructor becomes:
   ```
   
     PeerConfiguration(Iterable<RaftPeer> peers, Iterable<RaftPeer> listeners) {
       this.peers = newMap(peers, "peers", Collections.emptyMap());
       this.listeners = newMap(listeners, "listeners", this.peers);
     }
   ```

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -145,6 +149,10 @@ boolean containsInConf(RaftPeerId peerId) {
     return conf.contains(peerId);
   }
 
+  boolean containsInListenerConf(RaftPeerId peerId) {

Review comment:
       Let's call it `containsListenerInConf`.
   
   Let's fix our terminology -- we only have Configuration or Conf, which has peers and listeners, but there is no such that as _Listener Conf_.  We do have listeners in a Conf.
   

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -172,6 +184,11 @@ boolean containsInBothConfs(RaftPeerId peerId) {
         (oldConf == null || containsInOldConf(peerId));
   }
 
+  boolean containsInBothListenerConfs(RaftPeerId peerId) {

Review comment:
       Rename it to `containsListenerInBothConfs`.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -163,6 +171,10 @@ boolean containsInOldConf(RaftPeerId peerId) {
     return oldConf != null && oldConf.contains(peerId);
   }
 
+  boolean containsInOldListenerConf(RaftPeerId peerId) {

Review comment:
       Rename it to `containsListenerInOldConf`.

##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
##########
@@ -36,9 +36,18 @@
    */
   RaftPeer getPeer(RaftPeerId id);
 
+  /**
+   * @return the listener corresponding to the given id;
+   *         or return null if the listener is not in this configuration.
+   */
+  RaftPeer getListener(RaftPeerId id);
+
   /** @return all the peers in the current configuration and the previous configuration. */
   Collection<RaftPeer> getAllPeers();
 
+  /** @return all the listeners in the current configuration and the previous configuration. */
+  Collection<RaftPeer> getListeners();

Review comment:
       Let's call it `getAllListeners()`.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -353,6 +356,12 @@ private void startInitializing() {
     // do not start FollowerState
   }
 
+  private void startAsListener() {
+    setRole(RaftPeerRole.LISTENER, "startAsListener");
+    //TODO(codings-dan): ListenerState
+    lifeCycle.transition(RUNNING);
+  }
+

Review comment:
       Let's don't change RaftServerImpl in this pull request.  We will change it later.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -196,6 +227,16 @@ public RaftPeer getPeer(RaftPeerId id) {
     return peers;
   }
 
+  @Override
+  public Collection<RaftPeer> getListeners() {
+    final Collection<RaftPeer> listeners = new ArrayList<>(conf.getListeners());
+    if (oldConf != null) {
+      oldConf.getListeners().stream().filter(p -> !listeners.contains(p))
+          .forEach(listeners::add);
+    }
+    return listeners;
+  }

Review comment:
       Let's refactor the code:
   ```
     @Override
     public Collection<RaftPeer> getAllPeers() {
       return getAll(PeerConfiguration::getPeers);
     }
   
     @Override
     public Collection<RaftPeer> getListeners() {
       return getAll(PeerConfiguration::getListeners);
     }
   
     private Collection<RaftPeer> getAll(Function<PeerConfiguration, Collection<RaftPeer>> getMethod) {
       final Collection<RaftPeer> peers = new ArrayList<>(getMethod.apply(conf));
       if (oldConf != null) {
         getMethod.apply(oldConf).stream()
             .filter(p -> !peers.contains(p))
             .forEach(peers::add);
       }
       return peers;
     }
   ```

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
##########
@@ -66,10 +84,18 @@ RaftPeer getPeer(RaftPeerId id) {
     return peers.get(id);
   }
 
+  RaftPeer getListener(RaftPeerId id) {
+    return listeners.get(id);
+  }
+
   boolean contains(RaftPeerId id) {
     return peers.containsKey(id);
   }
 
+  boolean containsInListener(RaftPeerId id) {

Review comment:
       Just call it `containsListener`.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
##########
@@ -186,6 +203,20 @@ public RaftPeer getPeer(RaftPeerId id) {
     return null;
   }
 
+  @Override
+  public RaftPeer getListener(RaftPeerId id) {
+    if (id == null) {
+      return null;
+    }
+    RaftPeer listener = conf.getListener(id);
+    if (listener != null) {
+      return listener;
+    } else if (oldConf != null) {
+      return oldConf.getListener(id);
+    }
+    return null;
+  }

Review comment:
       Let's refactor the code.
   ```
     @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) {
       if (id == null) {
         return null;
       }
       final RaftPeer peer = getMethod.apply(conf, id);
       if (peer != null) {
         return peer;
       } else if (oldConf != null) {
         return getMethod.apply(oldConf, id);
       }
       return null;
     }
   ```




-- 
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 change in pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan commented on a change in pull request #598:
URL: https://github.com/apache/ratis/pull/598#discussion_r809707037



##########
File path: ratis-server-api/src/main/java/org/apache/ratis/server/RaftConfiguration.java
##########
@@ -36,9 +36,18 @@
    */
   RaftPeer getPeer(RaftPeerId id);
 
+  /**
+   * @return the listener corresponding to the given id;
+   *         or return null if the listener is not in this configuration.
+   */
+  RaftPeer getListener(RaftPeerId id);
+
   /** @return all the peers in the current configuration and the previous configuration. */
   Collection<RaftPeer> getAllPeers();
 
+  /** @return all the listeners in the current configuration and the previous configuration. */
+  Collection<RaftPeer> getListeners();

Review comment:
       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] szetszwo commented on pull request #598: RATIS-1301. Support listener in RaftConfiguration

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1046868511


   @codings-dan, we may also think about how to  change SetConfigurationRequest and LeaderStateImpl.startSetConfiguration to support listeners. 


-- 
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 #598: RATIS-1301. Support listener in ratis

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


   > @codings-dan , actually, it is an old test but not the newly added test. Let's see if we can reproduce the failure.
   
   I will have a try.


-- 
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 edited a comment on pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
codings-dan edited a comment on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1044146744


   @szetszwo It seems that the stepdown leader related work may have bug, the previous problem did not reproduce, but a new problem appeared. [https://github.com/apache/ratis/runs/5244723553?check_suite_focus=true](url)


-- 
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 #598: RATIS-1301. Support listener in ratis

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


   


-- 
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 pull request #598: RATIS-1301. Support listener in ratis

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1037982892


   @codings-dan , thanks for working on this.  The change is quite big.  Let's separate it into two or more JIRAs.
   
   The first one could be changing RaftConfiguration and the related classes to support Listener.  This probably is the most important change.  We should think the API and data structure change in details.  It seems to me that we should not change RaftGroup at all since RaftGroup should not care about the roles of the peers.


-- 
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 pull request #598: RATIS-1301. Support listener in RaftConfiguration

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #598:
URL: https://github.com/apache/ratis/pull/598#issuecomment-1046867967


   @codings-dan , the next task could be removing the old APIs below and replacing them with the new API.
   ```
   +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
   @@ -63,10 +63,6 @@ class PeerConfiguration {
        return Collections.unmodifiableMap(map);
      }
    
   -  PeerConfiguration(Iterable<RaftPeer> peers) {
   -    this(peers, Collections.emptyList());
   -  }
   -
      PeerConfiguration(Iterable<RaftPeer> peers, Iterable<RaftPeer> listeners) {
        this.peers = newMap(peers, "peers", Collections.emptyMap());
        this.listeners = newMap(listeners, "listeners", this.peers);
   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 4bab3650..b46a2fbd 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
   @@ -63,10 +63,6 @@ final class RaftConfigurationImpl implements RaftConfiguration {
          return this;
        }
    
   -    Builder setConf(Iterable<RaftPeer> peers) {
   -      return setConf(new PeerConfiguration(peers));
   -    }
   -
        Builder setConf(Iterable<RaftPeer> peers, Iterable<RaftPeer> listeners) {
          return setConf(new PeerConfiguration(peers, listeners));
        }
   ```
   


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