File an issue to track it. I may look into it deeply later. https://issues.apache.org/jira/browse/RATIS-1714
Best, tison. tison <[email protected]> 于2022年10月1日周六 18:42写道: > Yes. Then it loses the most important reason to use Ratis, lol. And I > noticed that my original purpose is not missing change events. This can be > implemented with a revision in each watch request. > > Thank you! > > Best, > tison. > > > Tsz Wo Sze <[email protected]> 于2022年10月1日周六 18:36写道: > >> > ... is it possible to submit a request to RaftServer locally, ...? >> >> When the Leader is local, then you can call the RaftServer directly. >> But what if the Leader is changed? Without using a Ratis client, your >> own client has to handle leader change, which is not very easy to do. >> >> Tsz-Wo >> >> >> On Sat, Oct 1, 2022 at 5:45 PM tison <[email protected]> wrote: >> >>> That is, I know the map's server and RatisServer run on the same >>> process, and try to bypass the serde and network process but locally submit >>> a request with Message. >>> >>> Best, >>> tison. >>> >>> >>> tison <[email protected]> 于2022年10月1日周六 17:41写道: >>> >>>> Another direction can be, is it possible to submit a request to >>>> RaftServer locally, and thus not need to pipe a request from replicated >>>> map's client, map's server (i.e. RatisClient), and RatisServer? >>>> >>>> Best, >>>> tison. >>>> >>>> >>>> tison <[email protected]> 于2022年9月30日周五 19:03写道: >>>> >>>>> Thank you. I'll check the technical details. >>>>> >>>>> For the application framework, I find a possible way to add an >>>>> external watch manager who gets notified from callbacks registered to the >>>>> state machine. Although, this way asks the client to establish one more >>>>> connection to the watch manager (even if in the same process, different >>>>> port). >>>>> >>>>> Best, >>>>> tison. >>>>> >>>>> >>>>> Tsz Wo Sze <[email protected]> 于2022年9月30日周五 18:30写道: >>>>> >>>>>> > However, still I don't know what "watch an index" actually means. >>>>>> How can I obtain the index in the first place? What condition I will >>>>>> receive a one-shot response from? >>>>>> >>>>>> A log index can be obtained by RaftClientReply.getLogIndex(), which >>>>>> returns the raft log index corresponding to a write request. The work >>>>>> flow >>>>>> is like below: >>>>>> >>>>>> 1. A client sends a (write) request and then gets back a >>>>>> RaftClientReply r -- at that moment the request may only have been >>>>>> replicated to a MAJORITY of servers as specified by the Raft Algorithm. >>>>>> >>>>>> 2. A user application may have a more strict replication >>>>>> requirement. For example, it may want the request to be replicated to >>>>>> ALL >>>>>> the servers, instead of a MAJORITY. It may call >>>>>> watch(r.getLogIndex(), ReplicationLevel.ALL) in order to watch for the >>>>>> replication ALL condition. >>>>>> >>>>>> We also have MAJORITY_COMMITTED and ALL_COMMITTED replication >>>>>> levels. COMMITTED means that the server's commit-index has been >>>>>> increased >>>>>> at least the watched index. >>>>>> >>>>>> Hope it helps. >>>>>> Tsz-Wo >>>>>> >>>>>> On Fri, Sep 30, 2022 at 5:43 PM tison <[email protected]> wrote: >>>>>> >>>>>>> Hi Tsz-Wo, >>>>>>> >>>>>>> Thanks for your reply! >>>>>>> >>>>>>> I think in this way I can implement a one-shot watcher. >>>>>>> >>>>>>> The similar functionality in etcd defines as: >>>>>>> >>>>>>> rpc Watch(stream WatchRequest) returns (stream WatchResponse); >>>>>>> >>>>>>> ... and this is why I use "full-duplex server-client communication" >>>>>>> in the subject - Is it possible for a Ratis client opens a connection >>>>>>> to do >>>>>>> something like this? >>>>>>> >>>>>>> > AsyncApi.watch >>>>>>> >>>>>>> Yes. IIRC I asked questions about this "watch" function years ago >>>>>>> and understand it's not "watch a key" :) >>>>>>> >>>>>>> However, still I don't know what "watch an index" actually means. >>>>>>> How can I obtain the index in the first place? What condition I will >>>>>>> receive a one-shot response from? >>>>>>> >>>>>>> Best, >>>>>>> tison. >>>>>>> >>>>>>> >>>>>>> Tsz Wo Sze <[email protected]> 于2022年9月30日周五 15:17写道: >>>>>>> >>>>>>>> Hi tison, >>>>>>>> >>>>>>>> Since Ratis server is asynchronous event-driven. We may implement >>>>>>>> "watch >>>>>>>> a key" using the AsyncApi [1]: >>>>>>>> 1. Client sends a "watch a key" request by client.a >>>>>>>> sync().sendReadOnly(..). >>>>>>>> 2. StateMachine won't complete the future until the key satisfies >>>>>>>> the watch condition. >>>>>>>> >>>>>>>> One problem is that the async read calls in Ratis are ordered. >>>>>>>> Thus, the other ordered async calls sent by the same client would >>>>>>>> not be completed before the "watch a key" request is completed. >>>>>>>> Of course, we may work around it by creating a new client. A better >>>>>>>> solution is to support unordered async read (a new feature) in >>>>>>>> Ratis. >>>>>>>> >>>>>>>> Note that Ratis already supports unordered AsyncApi.watch(..), >>>>>>>> which can watch for the replication level of the given log index. >>>>>>>> Therefore, it is easy to add the new feature for supporting >>>>>>>> unordered async read. >>>>>>>> >>>>>>>> What do you think? >>>>>>>> >>>>>>>> Tsz-Wo >>>>>>>> >>>>>>>> [1] >>>>>>>> https://github.com/apache/ratis/blob/master/ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java >>>>>>>> >>>>>>>> >>>>>>>> On Fri, Sep 30, 2022 at 2:21 PM tison <[email protected]> wrote: >>>>>>>> >>>>>>>>> Hi, >>>>>>>>> >>>>>>>>> I'm trying to write a replicated map based on Ratis. >>>>>>>>> >>>>>>>>> One thing that blocks me here is that I'm trying to implement >>>>>>>>> something like "watch a key". While Ratis support basic RPC between >>>>>>>>> server >>>>>>>>> and client, it's a one-shot RPC call. I have two ideas here but both >>>>>>>>> seems >>>>>>>>> not easy to implement: >>>>>>>>> >>>>>>>>> 1. Take the client connection and send back key changes even if >>>>>>>>> the client doesn't round-robin query it. However, the Rpc details in >>>>>>>>> under >>>>>>>>> encapsulation and never intend to be used. >>>>>>>>> 2. Wrapper an RMap server over the Ratis server. However, in this >>>>>>>>> case, It's the RMap server that should be responsible for initiating >>>>>>>>> and >>>>>>>>> managing the connection. Ratis server encapsulates this detail and >>>>>>>>> the only >>>>>>>>> way I can imagine is the RMap server as a proxy, but it's quite >>>>>>>>> clumsy to >>>>>>>>> have one more hop. >>>>>>>>> >>>>>>>>> Looking forward to your ideas. >>>>>>>>> >>>>>>>>> Best, >>>>>>>>> tison. >>>>>>>>> >>>>>>>>
