ratis
ratis copied to clipboard
RATIS-1557. Linearizable Leader Read-Only Request Implementation
What changes were proposed in this pull request?
Add linearizable read-only implementation for Raft leader. See https://issues.apache.org/jira/secure/attachment/13047304/13047304_Proposal-Support+leader+readIndex.pdf for my original proposal.
This PR only includes the Leader side implementation. Follower side will be implemented afterwards.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1557
How was this patch tested?
It is currently a draft, so I just add a simple unit test. I'm going to add more unit tests to verify the correctness of implementation soon.
@szetszwo @codings-dan This PR is a draft for Linearizable Reads, please take a look. Any suggestions on overall design or code details would be great!
@SzyWilliam , thanks for working on this! Will do the review later today.
@szetszwo I've made corresponding changes. Also I added some doc in getReadIndex() and readOnlyAsync().
@szetszwo, Thanks for the review.
- Raft section 6.4 says a single round of heartbeats can answer any number of read-only requests that accumulate, so we may not have to assign one-one mapping between read only request with heartbeats.
- Indeed batching read only requests is a great idea.
- Should we add a method to trigger heartbeat manually? A possible solution might be: When the first read-only request arrives, we manually trigger a heartbeat. Any subsequent read-only requests will only be added to pending queue and be answered by this heartbeat. What do you think?
@SzyWilliam , before triggering a heartbeat, it should record the current call id. When it receives a reply, it check if the reply have a later call id. Then, it can make sure, the reply comes after the trigger. Let me file a JIRA to so.
@szetszwo I fixed the review issues. Please take a look at the code again. Thanks!
Should we add a method to trigger heartbeat manually?
Yes, we probably should add a triggerHeartbeat(..) method in LogAppender.
- Indeed batching read only requests is a great idea.
For batching the read requests, we may use the commit index as the key so that the read requests with the same commit index will get the same heartbeat future. Then, the number of triggered heartbeat will be bounded by the number of commit indices. It won't have too many heartbeats then.
@szetszwo Thanks for the review! I'll add triggerHeartbeat
& batching mechanism after https://github.com/apache/ratis/pull/706 is merged.
@SzyWilliam , since this is a complicated change, could you split this into smaller subtasks? We could have
trigger a heartbeat immediately;
boardcast a heartbeat;
wait for the applied index.
@szetszwo OK, I'll split it into subtasks.
This PR will be split into subtasks, so I'll just close it now.