[ 
https://issues.apache.org/jira/browse/RATIS-386?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16685836#comment-16685836
 ] 

Tsz Wo Nicholas Sze commented on RATIS-386:
-------------------------------------------

Thanks [~shashikant].  Some comments on the patch:
- remove retryInterval from RaftClientImpl.  Just use the sleep time in 
retryPolicy.
-* Then, we can remove RETRY_INTERVAL_KEY and the related fields from 
RaftClientConfigKeys
- retryCount should be volatile.
-* BTW, retryCount seems misleading.  It is not clear whether the first attempt 
is included.  How about rename it to attemptCount?  If you agree, let's also 
rename the use of "retryCount" in RetryPolicy.
- Let's move the retryPolicy check to sendRequestAsync(..); see below.
-* The change of SlidingWindow is not required.
-* Use RaftException (or create a new subclass) instead of TimeoutExceptoin.

{code}
@@ -268,10 +271,10 @@ final class RaftClientImpl implements RaftClient {
       PendingAsyncRequest pending) {
     final RaftClientRequest request = pending.newRequest();
     final CompletableFuture<RaftClientReply> f = pending.getReplyFuture();
-    return sendRequestAsync(request).thenCompose(reply -> {
+    return sendRequestAsync(request, 
pending.getRetryCount()).thenCompose(reply -> {
       if (reply == null) {
-        LOG.debug("schedule a retry in {} for {}", retryInterval, request);
-        scheduler.onTimeout(retryInterval,
+        LOG.debug("schedule attempt #{} with policy {} for {}", 
pending.getRetryCount(), retryPolicy, request);
+        scheduler.onTimeout(retryPolicy.getSleepTime(),
             () -> getSlidingWindow(request).retry(pending, 
this::sendRequestWithRetryAsync),
             LOG, () -> "Failed to retry " + request);
       } else {
@@ -301,12 +304,17 @@ final class RaftClientImpl implements RaftClient {
     }
   }
 
-  private CompletableFuture<RaftClientReply> sendRequestAsync(
-      RaftClientRequest request) {
+  private CompletableFuture<RaftClientReply> 
sendRequestAsync(RaftClientRequest request, int attemptCount) {
     LOG.debug("{}: send* {}", clientId, request);
     return clientRpc.sendRequestAsync(request).thenApply(reply -> {
       LOG.debug("{}: receive* {}", clientId, reply);
       reply = handleNotLeaderException(request, reply);
+      if (reply == null) {
+        if (!retryPolicy.shouldRetry(attemptCount)) {
+          reply = new RaftClientReply(request, new RaftException(
+              "Failed " + request + " for " + attemptCount + " attempts with " 
+ retryPolicy), null);
+        }
+      }
       if (reply != null) {
         getSlidingWindow(request).receiveReply(
{code}


> Raft Client Async API's should honor Retry Policy 
> --------------------------------------------------
>
>                 Key: RATIS-386
>                 URL: https://issues.apache.org/jira/browse/RATIS-386
>             Project: Ratis
>          Issue Type: Improvement
>          Components: client
>    Affects Versions: 0.3.0
>            Reporter: Shashikant Banerjee
>            Assignee: Shashikant Banerjee
>            Priority: Major
>             Fix For: 0.3.0
>
>         Attachments: RATIS-386.000.patch
>
>
> Raft client sync Api has support for retry policies. Similarly, for Async 
> API's including watch Api, support for Retry Policy is required.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to