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

Ivan Andika commented on RATIS-2350:
------------------------------------

[~szetszwo] Thanks for the feedback. However, the idea of read-your-own-write 
requires the system (client or server) some kind of "session" or "client" to 
keep track of the last write of the client (like the state ID in HDFS observer 
read). However since AWS S3 seems to interact using stateless HTTP requests (I 
researched before that AWS S3 request does not seem to have a client ID, only 
request ID), the read-after-write consistency is that all users should see the 
previous writes (not only the client that writes the object).

>From [https://aws.amazon.com/s3/consistency/:] "After a successful write of a 
>new object, or an overwrite or delete of an existing object, any subsequent 
>read request immediately receives the latest version of the object." seems to 
>point that any HTTP GET request afterwards, no matter from which client will 
>see the latest object.

>From 
>[https://docs.aws.amazon.com/AmazonS3/latest/userguide/Welcome.html#ConsistencyModel]

!screenshot-2.png|width=685,height=277!

Since it's strongly consistent, the Client 1's Read 1 should see the latest 
write 2 "color = ruby". However, for read-your-own-write, Read 1 can simply 
return "color = red" from the client 1's Write 1 and does not need to see 
Client 2's Write 2.

That's why it was a bit confusing where "read-after-write" can mean strong 
consistency or "read-your-own-write" consistency.

Another ref is also: 
https://www.allthingsdistributed.com/2021/04/s3-strong-consistency.html

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