szetszwo commented on code in PR #1275:
URL: https://github.com/apache/ratis/pull/1275#discussion_r2193149374


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -819,23 +819,25 @@ private CompletableFuture<RaftClientReply> 
appendTransaction(
 
     assertLifeCycleState(LifeCycle.States.RUNNING);
 
+    final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
+    final PendingRequests.Permit permit = 
leaderState.tryAcquirePendingRequest(request.getMessage());
+    if (permit == null) {
+      cacheEntry.failWithException(new ResourceUnavailableException(
+          getMemberId() + ": Failed to acquire a pending write request for " + 
request));
+      return cacheEntry.getReplyFuture();
+    }
+
     final PendingRequest pending;
     synchronized (this) {
       final CompletableFuture<RaftClientReply> reply = 
checkLeaderState(request, cacheEntry);
       if (reply != null) {
+        leaderState.cancelPendingRequest(permit, request.getMessage());
         return reply;
       }
 
       // append the message to its local log
-      final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
       writeIndexCache.add(request.getClientId(), context.getLogIndexFuture());
 
-      final PendingRequests.Permit permit = 
leaderState.tryAcquirePendingRequest(request.getMessage());
-      if (permit == null) {
-        cacheEntry.failWithException(new ResourceUnavailableException(
-            getMemberId() + ": Failed to acquire a pending write request for " 
+ request));
-        return cacheEntry.getReplyFuture();
-      }

Review Comment:
   Get LeaderState again and check if it is the same.
   
   BTW, there is an existing bug that `writeIndexCache.add(..)` should be 
called after having a permit.
   ```diff
   +    final LeaderStateImpl leaderState;
        final PendingRequest pending;
        synchronized (this) {
   ```
   ```java
         leaderState = role.getLeaderStateNonNull();
         final PendingRequests.Permit permit = leaderState == 
unsyncedLeaderState? unsyncedPermit
             : leaderState.tryAcquirePendingRequest(request.getMessage());
         if (permit == null) {
           return getResourceUnavailableReply(request, cacheEntry);
         }
   
         // append the message to its local log
         writeIndexCache.add(request.getClientId(), 
context.getLogIndexFuture());
   ```



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -819,23 +819,25 @@ private CompletableFuture<RaftClientReply> 
appendTransaction(
 
     assertLifeCycleState(LifeCycle.States.RUNNING);
 
+    final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
+    final PendingRequests.Permit permit = 
leaderState.tryAcquirePendingRequest(request.getMessage());
+    if (permit == null) {
+      cacheEntry.failWithException(new ResourceUnavailableException(
+          getMemberId() + ": Failed to acquire a pending write request for " + 
request));
+      return cacheEntry.getReplyFuture();
+    }
+
     final PendingRequest pending;
     synchronized (this) {
       final CompletableFuture<RaftClientReply> reply = 
checkLeaderState(request, cacheEntry);
       if (reply != null) {
+        leaderState.cancelPendingRequest(permit, request.getMessage());

Review Comment:
   When `reply != null`, the server is no longer the leader. So, we don't have 
to cancal the permit.
   
   BTW, we should move `assertGroup(..)`
   - from checkLeaderState(RaftClientRequest, CacheEntry)
   - to checkLeaderState(RaftClientRequest)
   
   since `writeAsyncImpl` already called checkLeaderState(RaftClientRequest).   
We don't have to call it again inside the synchronized block.
   
   ```diff
   @@ -752,6 +752,11 @@ class RaftServerImpl implements RaftServer.Division,
      }
    
      private CompletableFuture<RaftClientReply> 
checkLeaderState(RaftClientRequest request) {
   +    try {
   +      assertGroup(getMemberId(), request);
   +    } catch (GroupMismatchException e) {
   +      return JavaUtils.completeExceptionally(e);
   +    }
        return checkLeaderState(request, null);
      }
    
   @@ -759,12 +764,6 @@ class RaftServerImpl implements RaftServer.Division,
       * @return null if the server is in leader state.
       */
      private CompletableFuture<RaftClientReply> 
checkLeaderState(RaftClientRequest request, CacheEntry entry) {
   -    try {
   -      assertGroup(getMemberId(), request);
   -    } catch (GroupMismatchException e) {
   -      return RetryCacheImpl.failWithException(e, entry);
   -    }
   -
        if (!getInfo().isLeader()) {
          NotLeaderException exception = generateNotLeaderException();
          final RaftClientReply reply = newExceptionReply(request, exception);
   ```



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -819,23 +819,25 @@ private CompletableFuture<RaftClientReply> 
appendTransaction(
 
     assertLifeCycleState(LifeCycle.States.RUNNING);
 
+    final LeaderStateImpl leaderState = role.getLeaderStateNonNull();
+    final PendingRequests.Permit permit = 
leaderState.tryAcquirePendingRequest(request.getMessage());

Review Comment:
   Let's call it `unsyncedPermit` since, if the leader state has changed, we 
need to get the permit again.
   ```java
       final PendingRequests.Permit unsyncedPermit = 
unsyncedLeaderState.tryAcquirePendingRequest(request.getMessage());
       if (unsyncedPermit == null) {
         return getResourceUnavailableReply(request, cacheEntry);
       }
   ```
   
   ```java
     private CompletableFuture<RaftClientReply> 
getResourceUnavailableReply(RaftClientRequest request, CacheEntry entry) {
       return entry.failWithException(new ResourceUnavailableException(
           getMemberId() + ": Failed to acquire a pending write request for " + 
request));
     }
   ```
   



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -819,23 +819,25 @@ private CompletableFuture<RaftClientReply> 
appendTransaction(
 
     assertLifeCycleState(LifeCycle.States.RUNNING);
 
+    final LeaderStateImpl leaderState = role.getLeaderStateNonNull();

Review Comment:
   We should use `getLeaderState()` and check nulll;
   ```java
       final LeaderStateImpl unsyncedLeaderState = 
role.getLeaderState().orElse(null);
       if (unsyncedLeaderState == null) {
         final RaftClientReply reply = newExceptionReply(request, 
generateNotLeaderException());
         return RetryCacheImpl.failWithReply(reply, cacheEntry);
       }
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to