HI tison, William,
Thanks for pointing out the problem!
For the long term, we should support Read-After-Write Consistency. It
seems easy to modify the read-index algorithm to support it: use the index
of the previous request as the read index, instead of the commit index.
As a work around, we may use the async API with thenApply:
- Client 1: async put k1 as v1 and get future f1
- Client 1: f1.thenApply(reply -> if (reply.isSucces()) {async get k1}).
Tsz-Wo
On Mon, Aug 28, 2023 at 8:28 AM William Song <[email protected]> wrote:
> Hi tison,
>
> The readIndex provides only the linearizable consistency guarantee. A
> linearizable read returned by RaftServer at least reflects any changes
> already committed and replied to the client. If you require a more strict
> consistency guarantee, like read reflecting the latest write, you may need
> to add extra synchronization logic. I think blocking IO or dependencies on
> completionStage will both work.
>
> Regards,
> William
>
>
> 2023年8月27日 12:20,tison <[email protected]> 写道:
>
> Hi,
>
> I have asked a related question before[1] about syncing better query
> and applytransaction.
>
> Now, when I dive deeper into the Raft-based system, I notice a new case
> for consistency.
>
> Take the following executions as an example (supposed we implement a Map
> statemachine):
>
> Client 1 put k1 as v1
> Client 1 get k1
>
> Generally, we expect the get requests gets v1. But if these two requests
> are issued in async, it's not always true.
>
> Even the client TCP connection can guarantee that "get k1" goes after "put
> k1", and we do read index for "get k1", the read index returned can be
> lower then "put k1" if it's not yet committed. Then the "get k1" request
> gets an empty value.
>
> I'm not sure what is the best practices to implement read my write for a
> Ratis-based system.
>
> 1. If we're using blocking IO, it won't be an issue cause "get k1" must be
> after "put k1" committed and even applied.
> 2. If we build an extra facade over Ratis like IoTDB does, we can check
> the read conditions as I did at [2].
> 3. But what if we only customize the statemachine and use the Ratis
> client? I don't know which happens before relations I can depend on for
> concurrent query and applytransaction.
>
> Also, in [2] for raft-rs there is one more subtle issue that their
> ReadIndex request can get lost or silently ignored, which causes the caller
> to retry that can even get a much later read index. I'm curious if Ratis
> has the same issue (lose ReadIndex or silently ignore and the caller
> doesn't get response forever).
>
> Best,
> tison.
>
> [1] https://lists.apache.org/thread/wh9cf9456y1k3pt9v0qs9o3wychl937s
> [2]
> https://github.com/tikv/raft-rs/discussions/525#discussioncomment-6819686
>
>
>