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


##########
ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java:
##########
@@ -169,8 +172,17 @@ static <C extends MiniRaftCluster> void 
runTestFollowerReadOnlyParallel(C cluste
         writeReplies.add(new Reply(count, 
leaderClient.async().send(WAIT_AND_INCREMENT)));
         Thread.sleep(100);
 
-        assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0));
-        f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, 
f1)));
+        if (readIndexType == Type.REPLIED_INDEX) {
+          // With REPLIED_INDEX the read index only advances after the leader 
has applied the
+          // transaction and the reply batch is flushed.  WAIT_AND_INCREMENT 
takes 500 ms in
+          // the state machine but we only waited 100 ms, so its reply has not 
been generated
+          // yet and the follower read may only see the preceding sync 
INCREMENT (count - 1).
+          assertReplyAtLeast(count - 1, f0Client.io().sendReadOnly(QUERY, f0));
+          f1Replies.add(new Reply(count - 1, 
f1Client.async().sendReadOnly(QUERY, f1)));
+        } else {
+          assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0));
+          f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, 
f1)));
+        }

Review Comment:
   Sorry that the repliedIndex idea may not work:
   
   We have the invariant 
   - repliedIndex <= appliedIndex <= committedIndex 
   
   The repliedIndex idea is: When repliedIndex < appliedIndex, it is safe for a 
follower to return any states >= repliedIndex since appliedIndex is not yet 
visible to any clients -- The data is not yet stale in this sense.
   
   However, after a follower/leader has returned a read call, its appliedIndex 
A becomes visible to that client.  The subsequence reads must return a state >= 
A.  Below is an example of the problem:
   1. Leader: repliedIndex = 10 < appliedIndex = 20
   2. Follower 1: appliedIndex = 18
   3. Follower 2: appliedIndex = 14
   4. Client first reads from Follower 1.
   5. The same client reads from Follower 2 (or the Leader) <---- stale read.
   
   Let's see if there are some ways to make it work.



-- 
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