Cross post that it's implemented at https://github.com/apache/ratis/pull/913
Best, tison. Tsz Wo Sze <[email protected]> 于2023年8月30日周三 02:03写道: > > The issue of thenApply is the same as blocking IO that we cannot > pipeline requests but only send the next read after the pioneer write > return from the server. > > Async thenApply is better then blocking IO in the sense that thenApply > only blocks the read requests requiring Read-After-Write Consistency, > while blocking IO will block all the requests. Consider the following > > - Client 1: async put k1 as v1 and get future f1 > - Client 1: f1.thenApply(reply -> if (reply.isSucces()) {async get k1}). > - Client 1: async put k2 as v2 > - Client 1: async read k3 > > The last two requests won't be blocked. > > > 1. Who drive the new read index process? > > 2. How to fulfill "the previous request's index" as read index? > > The leader should drive the new read index process. When a read request > requires Read-After-Write Consistency, the request will get the log index > of the last write request of the same client and use it as the read index. > It seems we need to add a CompletableFuture<long> logIndexFuture. > > The followers are irrelevant since they do not have support write. > > > I’ll fire a issue if we plan to implement it. > > Sure, let's do it. :) > > Tsz-Wo > > On Tue, Aug 29, 2023 at 12:16 AM William Song <[email protected]> wrote: > >> > 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. >> >> I’ll fire a issue if we plan to implement it. >> >> > We can resolve this issue if we use Ratis as IoTDB that wraps the >> RaftServer with a facade and handles server-side connection customizedly. >> That is, the client pipelines requests, but the server sequences the >> requests and `submitClientRequest` with thenApply chain. >> >> IoTDB’s RaftServer facade is more like a customized client, rather than a >> server. It pipelines the caller’s requests and then forwards them to the >> Raft server sequentially. >> >> I think a `Connection` abstract inside which each request casually >> depends on all the previously requests is what you are asking for. A >> potential workaround is to implement a `Connection` which accepts async >> call but use `thenApply` internally to arrange these async calls. It can >> be officially implemented into Ratis if it’s a common demand. >> >> Hope it helps, >> William >> >> >> >> >> 2023年8月29日 07:14,tison <[email protected]> 写道: >> >> > use the index of the previous request as the read index, instead of >> the commit index >> >> Yes. This is exactly what we semantically want to do. Here are two >> questions we need to answer: >> >> 1. Who drive the new read index process? >> 2. How to fulfill "the previous request's index" as read index? >> >> They can be the same problem. >> >> If the client drives the process, then the only way it knows "the >> previous request's index" is waiting for the previous request return. This >> is no better than blocking IO or thenApply. >> >> If the server drives the process, then "the previous request" means the >> previous request of "this connection". I don't find a connection >> abstraction in the codebase and this is what I encounter issues on the >> first place: query/applyTransaction accept requests from all the clients >> synchronously and not even in order to when they arrive the server. >> >> Best, >> tison. >> >> >> tison <[email protected]> 于2023年8月29日周二 07:00写道: >> >>> Thank Wiliam & Tzs Wo! >>> >>> Yeah I just realized that it's somehow a "client-side" or >>> "connection-aspect" issue. >>> >>> The issue of thenApply is the same as blocking IO that we cannot >>> pipeline requests but only send the next read after the pioneer write >>> return from the server. >>> >>> We can resolve this issue if we use Ratis as IoTDB that wraps the >>> RaftServer with a facade and handles server-side connection customizedly. >>> That is, the client pipelines requests, but the server sequences the >>> requests and `submitClientRequest` with thenApply chain. This is like how >>> ZooKeeper's CommitProcessor chains its read/write requests[1]. >>> >>> So here we may have another perspective on the question: what are the >>> expected methods of application based on Ratis? >>> >>> If we want users to write applications with RaftClient in App's >>> client-side and only customized StateMachine on App's server side, even use >>> RaftServer as the server facade, then the current query/applyTransaction, >>> and even the fixed one-shot RPC[2] can be a limitation. >>> >>> If we encourage users to wrap RaftServer with their own server facade >>> and use RaftClient in the App's server-side on demand like IoTDB does, then >>> maybe the experience of submitting requests on the server side can be >>> improved; or we provide some examples to do so. (Generally, it should be >>> fine to always use RaftClient in the App's server-side, while it can cause >>> one extra codec, but we already have to send the request to other peers on >>> writes.) >>> >>> Best, >>> tison. >>> >>> [1] >>> https://github.com/apache/zookeeper/blob/bc0e61854ca362261d899295422ac14dd5d82e59/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java#L251-L259 >>> [2] https://lists.apache.org/thread/rg65qoph54hlpdhmoc3p80sd0z6wzwjm >>> >>> Tsz Wo Sze <[email protected]> 于2023年8月29日周二 02:36写道: >>> >>>> 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 >>>>> >>>>> >>>>> >>
