[ 
https://issues.apache.org/jira/browse/RATIS-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz-wo Sze updated RATIS-2350:
------------------------------
    Description: 
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.

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

  was:
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:
bq. Assertion: _read-after-write is more consistent than linearizable read_
Recall the definitions:
bq. read-after-write: Within the same client, the read called after write must 
able to see the change of the write
bq. linearizable read: the read is linearizable (i.e. it won't read stale data)
Suppose readIndex is 9 and writeIndex is 10.  By definition read-after-write 
must return any state at log index A after 10 while linearizable read must 
return any state at log index L after 9.  The assertion incorrectly check if A 
>= L, which is not a requirement.



-----
Original Summary: TestReadOnlyRequestWithGrpc may fail intermittently

Original Description:
{code}
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].



> Fix readAfterWrite bugs
> -----------------------
>
>                 Key: RATIS-2350
>                 URL: https://issues.apache.org/jira/browse/RATIS-2350
>             Project: Ratis
>          Issue Type: Bug
>          Components: gRPC
>            Reporter: Tsz-wo Sze
>            Assignee: Tsz-wo Sze
>            Priority: Major
>
> 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.
> ----
> 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