This is an automated email from the ASF dual-hosted git repository.

williamsong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new e306b0290 RATIS-1943. Pending read requests instead of failing when 
leader not ready (#973)
e306b0290 is described below

commit e306b0290270170d17dfc27730f305b4b89276cf
Author: William Song <[email protected]>
AuthorDate: Fri Dec 1 11:53:39 2023 +0800

    RATIS-1943. Pending read requests instead of failing when leader not ready 
(#973)
---
 .../java/org/apache/ratis/server/impl/LeaderStateImpl.java | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
index e1b12d486..565cb116c 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
@@ -306,6 +306,10 @@ class LeaderStateImpl implements LeaderState {
     /** This future will be completed after the log entry is applied. */
     private final CompletableFuture<Long> appliedIndexFuture = new 
CompletableFuture<>();
 
+    CompletableFuture<Long> getAppliedIndexFuture() {
+      return appliedIndexFuture;
+    }
+
     boolean isApplied(LogEntryProto logEntry) {
       if (appliedIndexFuture.isDone()) {
         return true;
@@ -444,9 +448,10 @@ class LeaderStateImpl implements LeaderState {
       LOG.warn("{}: Caught exception in sendNotLeaderResponses", this, e);
     }
     messageStreamRequests.clear();
-    // TODO client should retry on NotLeaderException
     readIndexHeartbeats.failListeners(nle);
     lease.getAndSetEnabled(false);
+    startupLogEntry.get().getAppliedIndexFuture().completeExceptionally(
+        new ReadIndexException("failed to obtain read index since: ", nle));
     server.getServerRpc().notifyNotLeader(server.getMemberId().getGroupId());
     logAppenderMetrics.unregister();
     raftServerMetrics.unregister();
@@ -1117,12 +1122,9 @@ class LeaderStateImpl implements LeaderState {
       return CompletableFuture.completedFuture(readIndex);
     }
 
-    // leader has not committed any entries in this term, reject
-    // TODO: wait for leader to become ready instead of failing the request.
+    // leader has not committed any entries in this term
     if (!isReady()) {
-      return JavaUtils.completeExceptionally(new ReadIndexException(
-          "Failed to getReadIndex " + readIndex + " since the term is not yet 
committed.",
-          new LeaderNotReadyException(server.getMemberId())));
+      return startupLogEntry.get().getAppliedIndexFuture();
     }
 
     // if lease is enabled, check lease first

Reply via email to