RATIS-2403. Support leader batch write to improve linearizable follower read throughput#1362
RATIS-2403. Support leader batch write to improve linearizable follower read throughput#1362ivandika3 wants to merge 10 commits intoapache:masterfrom
Conversation
…er read throughput
Generated-by: Cursor
Generated-by: Cursor
…t swap, add stopReplyFlusher to stop Generated-by: Cursor
| if (readIndexType == Type.REPLIED_INDEX) { | ||
| // With REPLIED_INDEX the read index only advances after the leader has applied the | ||
| // transaction and the reply batch is flushed. WAIT_AND_INCREMENT takes 500 ms in | ||
| // the state machine but we only waited 100 ms, so its reply has not been generated | ||
| // yet and the follower read may only see the preceding sync INCREMENT (count - 1). | ||
| assertReplyAtLeast(count - 1, f0Client.io().sendReadOnly(QUERY, f0)); | ||
| f1Replies.add(new Reply(count - 1, f1Client.async().sendReadOnly(QUERY, f1))); | ||
| } else { | ||
| assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0)); | ||
| f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, f1))); | ||
| } |
There was a problem hiding this comment.
I tried to fix this failing test, but the issue is that runTestFollowerReadOnlyParallel does not wait for the WAIT_AND_INCREMENT (which delays 500 ms although the test only sleep for 100ms) to return so technically we cannot assert that the count is incremented by WAIT_AND_INCREMENT since the StateMachine#applyTransactions future has not finished yet. I might be wrong.
When I tried to make repliedIndex to be current ServerState appliedIndex instead of the highest index of all the replies, the test pass. However, I think appliedIndex should be higher than repliedIndex so the follower might still need to wait longer than necessary. I considered to use the StateMachine.lastAppliedTermIndex, but we cannot guarantee the correctness or semantics of StateMachine.lastAppliedTermIndex (i.e. is it the highest index where applyTransaction future actually finish or simply the last applyTransaction called). I also found out that ServerState.appliedIndex does not mean that the async StateMachine#applyTransaction futures actually finish (although the sync StateMachine#applyTransactionSerial is guaranteed to be finished).
There was a problem hiding this comment.
Sorry that the repliedIndex idea may not work:
We have the invariant
- repliedIndex <= appliedIndex <= committedIndex
The repliedIndex idea is: When repliedIndex < appliedIndex, it is safe for a follower to return any states >= repliedIndex since appliedIndex is not yet visible to any clients -- The data is not yet stale in this sense.
However, after a follower/leader has returned a read call, its appliedIndex A becomes visible to that client. The subsequence reads must return a state >= A. Below is an example of the problem:
- Leader: repliedIndex = 10 < appliedIndex = 20
- Follower 1: appliedIndex = 18
- Follower 2: appliedIndex = 14
- Client first reads from Follower 1.
- The same client reads from Follower 2 (or the Leader) <---- stale read.
Let's see if there are some ways to make it work.
szetszwo
left a comment
There was a problem hiding this comment.
@ivandika3 , thanks a lot for working on this!
Please https://issues.apache.org/jira/secure/attachment/13081003/1362_review.patch for my review. I did not put all the comments here.
| private final boolean readIndexAppliedIndexEnabled; | ||
| private final RaftServerConfigKeys.Read.ReadIndex.Type readIndexType; | ||
| private final Supplier<Long> readIndexSupplier; | ||
| private final MemoizedSupplier<String> readIndexLogPrefixSupplier; |
There was a problem hiding this comment.
Remove this field and use readIndexType.
| } | ||
| LOG.debug("readIndex={} ({}Index={}, readAfterWriteConsistentIndex={})", | ||
| readIndex, readIndexAppliedIndexEnabled ? "applied" : "commit", | ||
| readIndex, readIndexLogPrefixSupplier.get(), |
There was a problem hiding this comment.
Use readIndexType:
LOG.debug("readIndex={} ({}={}, readAfterWriteConsistentIndex={})",
readIndex, readIndexType, index, readAfterWriteConsistentIndex);| /** The interval at which held write replies are flushed. */ | ||
| private final TimeDuration repliedIndexBatchInterval; | ||
| /** The highest log index for which a write reply has been flushed (sent to the client). */ | ||
| private final AtomicLong repliedIndex; | ||
| /** Guards {@link #heldReplies}. */ | ||
| private final Object heldRepliesLock = new Object(); | ||
| /** Buffer holding write replies waiting to be flushed. Guarded by {@link #heldRepliesLock}. */ | ||
| private List<HeldReply> heldReplies = new ArrayList<>(); | ||
| /** Daemon thread that periodically flushes held replies. */ | ||
| private volatile Daemon replyFlusher; |
There was a problem hiding this comment.
Since there are quite many fields/methods/classes added, let's move all of them to a new class, say ReplyFlusher.
There was a problem hiding this comment.
Thanks for the idea and the patch. Updated based on the patch.
|
Regarding to this problem, one way to fix the bug is to hold also the read replies -- when a server's (leader or follower) appliedIndex > leader's repliedIndex, hold the read replies until leader's repliedIndex >= the local appliedIndex. The previous example becomes:
Note that Step 8 is not a stale read since the two reads can be considered as being sent in parallel and the read from F2 is faster than the read from F1. @ivandika3 , what do you think? |
|
@szetszwo Thanks for the insight about the issue and the possible idea to fix this. IMO storing the read replies implementation is going to be quite involved (which might hide other subtle correctness issue) and there might be some overhead in managing the pending read replies. The way I see it is that the current repliedIndex (leader batch implementation) has two main benefits:
How about we simply update the |
@ivandika3 , Even
The original readIndex algorithm has the invariant:
When a follower receives a read request, it cannot get a readIndex (= leader's committedIndex) > its appliedIndex at that time. There are two options to fix it:
|
What changes were proposed in this pull request?
This patch is to implement leader batch write as suggested by @szetszwo to by making leader delay some requests to ensure that ReadIndex increase in coarser, more discrete steps (the original ReadIndex returns the latest commitIndex which can increase rapidly for a busy server).
The main insight is that linearizability read requires the latest read to see all the replied writes, not writes that have been applied to the state machine (although in Raft replied writes implies that it has been applied in state machine). We can exploit this by withholding the write replies in a batch which is flushed (i.e. replies are sent) at a configurable interval. The idea is similar to "group commit" in other systems (although group commit is about batch write operations to a single IO operation).
These are the introduced implementations
RepliedIndex) which denotes the latest index of the replied write requests. Also introduceraft.server.read.read-index.type(as suggested by @szetszwo) since now we support 3 ReadIndex types.AtomicReference<List<HeldReply>>see 4a8a402, but the implementation onholdReplyis not atomic sinceAtomicReferenceis not designed for atomic list appends.ConcurrentLinkedQueuebut I am worried that if the push throughput is higher than poll throughput theflushRepliesmight be stuck polling. One solution is to get the queue size and only poll until that size, butConcurrentLinkedQueuesizemethod is not constant-time (refer to the Javadoc).LinkedList, checked with LLM and this is the response (it makes sense to me)clearand copying the list which can be costly)What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2403
How was this patch tested?
UT.