allow raft apply committed logs before they are persisted#537
Conversation
Signed-off-by: glorv <[email protected]>
7d8dc02 to
51a0ee5
Compare
|
@Connor1996 @gengliqi @overvenus @tonyxuqqi @BusyJay PTAL, thank you~ |
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
|
Perhaps we could organize and link a top-down design document or RFC to illustrate the specific changes in tikv/raftstore, interface impacts, etc., after removing the implicit code constraints related to apply entry, as well as the corresponding design in raft-rs. I remember @gengliqi has some relevant design documents before? |
| max_committed_size_per_ready: c.max_committed_size_per_ready, | ||
| }, | ||
| }; | ||
| r.raft_log.apply_unpersisted_log_limit = c.apply_unpersisted_log_limit; |
There was a problem hiding this comment.
pass it through RaftLog::new?
Signed-off-by: glorv <[email protected]>
| fn next_entries_upper_bound(&self) -> u64 { | ||
| std::cmp::min( | ||
| self.committed, | ||
| self.persisted + self.apply_unpersisted_log_limit, |
There was a problem hiding this comment.
how about passing log_limit from outside, like how max_size does. Then you can store the config inside RaftCore
|
@glorv We should have a RFC document to summarize the design. |
|
|
||
| /// Maximum raft log number that can be applied after commit but before persist. | ||
| /// The default value is 0, which means apply after both commit and persist. | ||
| pub apply_unpersisted_log_limit: u64, |
There was a problem hiding this comment.
Will it support dynamic config update?
There was a problem hiding this comment.
Yes, we provide a public function Raft::set_apply_unpersisted_log_limit to change this value in realtime.
|
|
||
| /// Maximum raft log number that can be applied after commit but before persist. | ||
| /// The default value is 0, which means apply after both commit and persist. | ||
| pub apply_unpersisted_log_limit: u64, |
There was a problem hiding this comment.
How about max_apply_unpersisted_log_limit?
| // 1. then it is possible idx > persisted. | ||
| // 2. when the application restart after applied but before committed entried(and committed index) is persisted | ||
| // then it is also possible idx > committed. | ||
| if idx < self.applied { |
There was a problem hiding this comment.
At least idx should be greater than or equal to self.committed.
There was a problem hiding this comment.
This is for the initialization phase, currently it call applied_to to set the applied_index. I will change it to keep this check for normal path.
| /// to apply to its state machine. | ||
| /// | ||
| /// Invariant: applied <= min(committed, persisted) | ||
| /// Invariant: applied <= min(committed, persisted + `apply_unpersisted_log_limit`) |
There was a problem hiding this comment.
apply_unpersisted_log_limit is a config that can be changed so it does not belong to Invariant.
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
|
@Connor1996 @gengliqi PTAL again, thanks~ |
Signed-off-by: glorv <[email protected]>
| /// on a quorum of nodes. | ||
| /// | ||
| /// Invariant: applied <= committed | ||
| /// NOTE: this invariant can be break after restart if `max_apply_unpersisted_log_limit` is 0, |
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
| /// the maximum log gap between persisted_index and applied_index. | ||
| /// Caller should ensure the value won't lead to the upper bound overflow. |
There was a problem hiding this comment.
The comment about overflow may be unnecessary. Alternatively, it can be written in Config.
| /// the maximum log gap between persisted_index and applied_index. | |
| /// Caller should ensure the value won't lead to the upper bound overflow. | |
| /// The maximum log gap between persisted and applied. |
Signed-off-by: glorv <[email protected]>
f66d2bb to
47c7f1d
Compare
|
I think we need to add more restrictions to these logs that can be applied before persisting so that correctness errors can be found. |
|
I will propose another pr to implement this restriction. |
ref: tikv/tikv#16457, tikv/tikv#16717
Design doc: tikv/rfcs#112
As
committedmeans more than quorum node are persisted, which no data loss even if all other node are down. So in this situation, it is safe to apply this log even if it is still not persisted.This PR introduces a new config
max_applied_unpersisted_log_limitthat allows return unpersisted raft log inlight_ready. This is one step to optimize the tail latency that one slow node can significantly impact the overall latency. By setting a proper value formax_applied_unpersisted_log_limitwe can avoid the in memory raft entries consumes too much memory which may lead to OOM.After this change, if
max_applied_unpersisted_log_limitis > 0, then it is possible: 1) applied > persisted. 2) applied > committed.(Only can happen at restart). So we loose some check since they are not always true anymore.