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

Ivan Andika edited comment on RATIS-2350 at 12/31/25 2:45 AM:
--------------------------------------------------------------

Thanks [~szetszwo] for the feedbacks.

> Actually, this is not hard to implement – just make the read wait for the 
> latest pending write to complete.

Thanks for the idea. Yes, it seems we need to keep track of pending writes and 
block read until the pending writes are completed.

> Is it a useful feature? What do you think?

 Currently, I'm simply trying to understand the Ratis consistency while I'm 
working on follower reads. I will think about it if needed.


was (Author: JIRAUSER298977):
Thanks [~szetszwo] for the feedbacks.

> Actually, this is not hard to implement – just make the read wait for the 
> latest pending write to complete.

Thanks for the idea. Yes, it seems we need to keep track of pending writes and 
block read until the pending writes are completed.

> Is it a useful feature? What do you think?

 Currently, I'm simply trying to understand the Ratis consistency while I'm 
working on follower reads.

> Fix readAfterWrite bugs
> -----------------------
>
>                 Key: RATIS-2350
>                 URL: https://issues.apache.org/jira/browse/RATIS-2350
>             Project: Ratis
>          Issue Type: Bug
>          Components: server
>            Reporter: Tsz-wo Sze
>            Assignee: Tsz-wo Sze
>            Priority: Major
>             Fix For: 3.3.0
>
>         Attachments: image-2025-12-31-10-28-28-475.png, screenshot-1.png, 
> screenshot-2.png
>
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> There are bugs in handling readAfterWrite requests:
>  # In LeaderStateImpl.getReadIndex(..), it should use the max of 
> readAfterWriteConsistentIndex and commitIndex.
>  # In WriteIndexCache.add(..), it should combine the current future with 
> previous future when the previous future exists.
> Improvement:
>  - Add lastAppliedIndex to ReadIndexQueue
>  - Replace Consumer<Long> with LongConsumer
> Bug in tests:
>  - In LinearizableReadTests.runTestReadAfterWrite(..), it tries to assert the 
> following:
> {quote}Assertion: _read-after-write is more consistent than linearizable read_
> {quote}
> Recall the definitions:
> {quote}Read-after-write: _Within the same client, the read called after write 
> must able to see the change of the write._
> {quote}
> {quote}Linearizable read: _The read is linearizable (i.e. it won't read stale 
> data)._
> {quote}
> Suppose readIndex is 9 and writeIndex is 10. By definition, read-after-write 
> must return any state at log index A >= 10 while linearizable read must 
> return any state at log index L >= 9. The assertion incorrectly check if A >= 
> L, which is not a requirement.  It is perfectly fine, for example, if A=11 < 
> L=12.
> ----
> Original Summary: TestReadOnlyRequestWithGrpc may fail intermittently
> Original Description:
> {code:java}
> org.apache.ratis.grpc.TestReadOnlyRequestWithGrpc.testReadAfterWrite -- Time 
> elapsed: 1.572 s <<< FAILURE!
> org.opentest4j.AssertionFailedError: expected: <true> but was: <false>
>       at 
> org.apache.ratis.ReadOnlyRequestTests.testReadAfterWriteImpl(ReadOnlyRequestTests.java:314)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:143)
>       at 
> org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:121)
>       at 
> org.apache.ratis.ReadOnlyRequestTests.testReadAfterWrite(ReadOnlyRequestTests.java:289)
> {code}
> It failed 8 in [this 10x10 
> run|https://github.com/apache/ratis/actions/runs/19023405871/job/54322726144].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to