You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by "CRZbulabula (via GitHub)" <gi...@apache.org> on 2023/08/22 14:13:10 UTC

[GitHub] [iotdb] CRZbulabula commented on a diff in pull request #10896: [IOTDB-6121] Consensus layer interface and exception handling refactoring

CRZbulabula commented on code in PR #10896:
URL: https://github.com/apache/iotdb/pull/10896#discussion_r1301660915


##########
iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/IConsensus.java:
##########
@@ -19,137 +19,179 @@
 
 package org.apache.iotdb.consensus;
 
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.consensus.common.Peer;
 import org.apache.iotdb.consensus.common.request.IConsensusRequest;
-import org.apache.iotdb.consensus.common.response.ConsensusGenericResponse;
-import org.apache.iotdb.consensus.common.response.ConsensusReadResponse;
-import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
 import org.apache.iotdb.consensus.exception.ConsensusException;
+import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException;
+import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException;
+import org.apache.iotdb.consensus.exception.IllegalPeerEndpointException;
+import org.apache.iotdb.consensus.exception.IllegalPeerNumException;
+import org.apache.iotdb.consensus.exception.PeerAlreadyInConsensusGroupException;
+import org.apache.iotdb.consensus.exception.PeerNotInConsensusGroupException;
 
 import javax.annotation.concurrent.ThreadSafe;
 
 import java.io.IOException;
 import java.util.List;
 
-/** Consensus module base class. */
+/** Consensus module base interface. */
 @ThreadSafe
 public interface IConsensus {
 
+  /**
+   * Start the consensus module.
+   *
+   * @throws IOException when start consensus errors
+   */
   void start() throws IOException;
 
+  /**
+   * Stop the consensus module.
+   *
+   * @throws IOException when stop consensus errors
+   */
   void stop() throws IOException;
 
-  // write API
-  ConsensusWriteResponse write(ConsensusGroupId groupId, IConsensusRequest request);
+  /**
+   * Write data to the corresponding consensus group.
+   *
+   * @param groupId the consensus group this request belongs
+   * @param request write request
+   * @return write result
+   * @throws ConsensusGroupNotExistException when the specified consensus group doesn't exist
+   * @throws ConsensusException when write doesn't success with other reasons
+   */
+  TSStatus write(ConsensusGroupId groupId, IConsensusRequest request) throws ConsensusException;
 
-  // read API
-  ConsensusReadResponse read(ConsensusGroupId groupId, IConsensusRequest request);
+  /**
+   * Read data from the corresponding consensus group.
+   *
+   * @param groupId the consensus group this request belongs
+   * @param request read request
+   * @return read result
+   * @throws ConsensusGroupNotExistException when the specified consensus group doesn't exist
+   * @throws ConsensusException when read doesn't success with other reasons
+   */
+  DataSet read(ConsensusGroupId groupId, IConsensusRequest request) throws ConsensusException;
 
   // multi consensus group API
 
   /**
    * Require the <em>local node</em> to create a Peer and become a member of the given consensus
    * group. This node will prepare and initialize local statemachine {@link IStateMachine} and other
-   * data structures. After this method returns, we can call {@link #addPeer(ConsensusGroupId,
+   * data structures. After this method returns, we can call {@link #addRemotePeer(ConsensusGroupId,
    * Peer)} to notify original group that this new Peer is prepared to be added into the latest
-   * configuration. createPeer should be called on a node that does not contain any peer of the
+   * configuration. createLocalPeer should be called on a node that does not contain any peer of the
    * consensus group, to avoid one node having more than one replica.
    *
-   * @param groupId the consensus group this Peer belongs
+   * @param groupId the consensus group this peer belongs
    * @param peers other known peers in this group
+   * @throws ConsensusGroupAlreadyExistException when the specified consensus group already exists
+   * @throws IllegalPeerNumException when the peer num is illegal. The exception is that it is legal
+   *     to pass an empty list for RaftConsensus
+   * @throws IllegalPeerEndpointException when peers don't contain local node. The exception is that
+   *     it is legal to pass an empty list for RaftConsensus
+   * @throws ConsensusException when createLocalPeer doesn't success with other reasons
    */
-  ConsensusGenericResponse createPeer(ConsensusGroupId groupId, List<Peer> peers);
+  void createLocalPeer(ConsensusGroupId groupId, List<Peer> peers) throws ConsensusException;
 
   /**
    * When the <em>local node</em> is no longer a member of the given consensus group, call this
    * method to do cleanup works. This method will close local statemachine {@link IStateMachine},
-   * delete local data and do other cleanup works. Be sure this method is called after successfully
-   * removing this peer from current consensus group configuration (by calling {@link
-   * #removePeer(ConsensusGroupId, Peer)} or {@link #changePeer(ConsensusGroupId, List)}).
+   * delete local data and do other cleanup works. deleteLocalPeer should be called after
+   * successfully removing this peer from current consensus group configuration (by calling {@link
+   * #removeRemotePeer(ConsensusGroupId, Peer)}).
    *
-   * @param groupId the consensus group this Peer used to belong
+   * @param groupId the consensus group this peer used to belong
+   * @throws ConsensusGroupAlreadyExistException when the specified consensus group doesn't exist

Review Comment:
   ```suggestion
      * @throws ConsensusGroupDoesNotExistException when the specified consensus group doesn't 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: reviews-unsubscribe@iotdb.apache.org

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