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/12/16 10:49:24 UTC

[GitHub] [ratis] codings-dan opened a new pull request, #798: RATIS-1759. Support client use linearizable read per request

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

   see https://issues.apache.org/jira/browse/RATIS-1759
   


-- 
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 #798: RATIS-1759. Support client use linearizable read per request

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


##########
ratis-client/src/main/java/org/apache/ratis/client/api/AsyncApi.java:
##########
@@ -45,15 +45,20 @@ default CompletableFuture<RaftClientReply> sendReadOnly(Message message) {
     return sendReadOnly(message, null);
   }
 
+  default CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server) {
+    return sendReadOnly(message, null, false);
+  }
+
   /**
    * Send the given readonly message asynchronously to the raft service.
    * Note that the reply futures are completed in the same order of the messages being sent.
    *
    * @param message The request message.
    * @param server The target server.  When server == null, send the message to the leader.
+   * @param readIndex weather use linearizable read
    * @return a future of the reply.
    */
-  CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server);
+  CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server, boolean readIndex);

Review Comment:
   Since the server parameter does not make sense for non-linearizable read, how about we add a new method instead?
   ```java
     /**
      * Send the given readonly message asynchronously to the raft service using non-linearizable read.
      * This method is useful when linearizable read is enabled
      * but this client prefers not using it for performance reason.
      * When linearizable read is disabled, this method is the same as {@link #sendReadOnly(Message)}. 
      *
      * @param message The request message.
      * @return a future of the reply.
      */
     CompletableFuture<RaftClientReply> sendReadOnlyNonLinearizable(Message message);
   ```



##########
ratis-proto/src/main/proto/Raft.proto:
##########
@@ -296,6 +296,10 @@ message ForwardRequestTypeProto {
 message ReadRequestTypeProto {
 }
 
+message ReadIndexRequestTypeProto {
+
+}

Review Comment:
   Let's add a field to ReadRequestTypeProto.
   ```proto
   message ReadRequestTypeProto {
     bool preferNonLinearizable = 1;
   }
   ```



-- 
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 merged pull request #798: RATIS-1759. Support client use linearizable read per request

Posted by GitBox <gi...@apache.org>.
codings-dan merged PR #798:
URL: https://github.com/apache/ratis/pull/798


-- 
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 #798: RATIS-1759. Support client use linearizable read per request

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


##########
ratis-client/src/main/java/org/apache/ratis/client/api/BlockingApi.java:
##########
@@ -45,14 +45,19 @@ default RaftClientReply sendReadOnly(Message message) throws IOException {
     return sendReadOnly(message, null);
   }
 
+  default RaftClientReply sendReadOnly(Message message, RaftPeerId server) throws IOException {
+    return sendReadOnly(message, server, false);
+  }
+
   /**
    * Send the given readonly message to the raft service.
    *
    * @param message The request message.
    * @param server The target server.  When server == null, send the message to the leader.
+   * @param readIndex weather use linearizable read.
    * @return the reply.
    */
-  RaftClientReply sendReadOnly(Message message, RaftPeerId server) throws IOException;
+  RaftClientReply sendReadOnly(Message message, RaftPeerId server, boolean readIndex) throws IOException;

Review Comment:
   just add a new interface, PTAL, thanks!



##########
ratis-client/src/main/java/org/apache/ratis/client/api/AsyncApi.java:
##########
@@ -45,15 +45,20 @@ default CompletableFuture<RaftClientReply> sendReadOnly(Message message) {
     return sendReadOnly(message, null);
   }
 
+  default CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server) {
+    return sendReadOnly(message, null, false);
+  }
+
   /**
    * Send the given readonly message asynchronously to the raft service.
    * Note that the reply futures are completed in the same order of the messages being sent.
    *
    * @param message The request message.
    * @param server The target server.  When server == null, send the message to the leader.
+   * @param readIndex weather use linearizable read
    * @return a future of the reply.
    */
-  CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server);
+  CompletableFuture<RaftClientReply> sendReadOnly(Message message, RaftPeerId server, boolean readIndex);

Review Comment:
   done



##########
ratis-proto/src/main/proto/Raft.proto:
##########
@@ -296,6 +296,10 @@ message ForwardRequestTypeProto {
 message ReadRequestTypeProto {
 }
 
+message ReadIndexRequestTypeProto {
+
+}

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] SzyWilliam commented on a diff in pull request #798: RATIS-1759. Support client use linearizable read per request

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


##########
ratis-client/src/main/java/org/apache/ratis/client/api/BlockingApi.java:
##########
@@ -45,14 +45,19 @@ default RaftClientReply sendReadOnly(Message message) throws IOException {
     return sendReadOnly(message, null);
   }
 
+  default RaftClientReply sendReadOnly(Message message, RaftPeerId server) throws IOException {
+    return sendReadOnly(message, server, false);
+  }
+
   /**
    * Send the given readonly message to the raft service.
    *
    * @param message The request message.
    * @param server The target server.  When server == null, send the message to the leader.
+   * @param readIndex weather use linearizable read.
    * @return the reply.
    */
-  RaftClientReply sendReadOnly(Message message, RaftPeerId server) throws IOException;
+  RaftClientReply sendReadOnly(Message message, RaftPeerId server, boolean readIndex) throws IOException;

Review Comment:
   Shall we use `Read.Option` as the third parameter? We may introduce lease-based linearizable read in the future, and boolean won't work then.



-- 
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 #798: RATIS-1759. Support client use linearizable read per request

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

   @szetszwo @SzyWilliam Thanks for helping review the pull request!


-- 
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 #798: RATIS-1759. Support client use linearizable read per request

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

   @szetszwo @SzyWilliam Could you help take a look at the pull request? 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