Skip to content

allow raft apply committed logs before they are persisted#537

Merged
gengliqi merged 14 commits into
tikv:masterfrom
glorv:apply-before-persist
Mar 28, 2024
Merged

allow raft apply committed logs before they are persisted#537
gengliqi merged 14 commits into
tikv:masterfrom
glorv:apply-before-persist

Conversation

@glorv

@glorv glorv commented Mar 11, 2024

Copy link
Copy Markdown
Contributor

ref: tikv/tikv#16457, tikv/tikv#16717

Design doc: tikv/rfcs#112

As committed means 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_limit that allows return unpersisted raft log in light_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 for max_applied_unpersisted_log_limit we can avoid the in memory raft entries consumes too much memory which may lead to OOM.

After this change, if max_applied_unpersisted_log_limit is > 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.

@glorv glorv force-pushed the apply-before-persist branch from 7d8dc02 to 51a0ee5 Compare March 11, 2024 03:30
@glorv

glorv commented Mar 11, 2024

Copy link
Copy Markdown
Contributor Author

@Connor1996 @gengliqi @overvenus @tonyxuqqi @BusyJay PTAL, thank you~

glorv added 5 commits March 11, 2024 13:03
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]>
@cfzjywxk

Copy link
Copy Markdown

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?

Comment thread src/raft.rs Outdated
max_committed_size_per_ready: c.max_committed_size_per_ready,
},
};
r.raft_log.apply_unpersisted_log_limit = c.apply_unpersisted_log_limit;

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

pass it through RaftLog::new?

Comment thread src/raft_log.rs
Signed-off-by: glorv <[email protected]>
Comment thread src/raft_log.rs Outdated
fn next_entries_upper_bound(&self) -> u64 {
std::cmp::min(
self.committed,
self.persisted + self.apply_unpersisted_log_limit,

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about passing log_limit from outside, like how max_size does. Then you can store the config inside RaftCore

@tonyxuqqi

Copy link
Copy Markdown

@glorv We should have a RFC document to summarize the design.

Comment thread src/config.rs Outdated

/// 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,

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will it support dynamic config update?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we provide a public function Raft::set_apply_unpersisted_log_limit to change this value in realtime.

Comment thread src/config.rs Outdated

/// 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,

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about max_apply_unpersisted_log_limit?

Comment thread src/raft_log.rs Outdated
// 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 {

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At least idx should be greater than or equal to self.committed.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Comment thread src/raft_log.rs Outdated
/// to apply to its state machine.
///
/// Invariant: applied <= min(committed, persisted)
/// Invariant: applied <= min(committed, persisted + `apply_unpersisted_log_limit`)

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

apply_unpersisted_log_limit is a config that can be changed so it does not belong to Invariant.

@glorv

glorv commented Mar 28, 2024

Copy link
Copy Markdown
Contributor Author

@Connor1996 @gengliqi PTAL again, thanks~

Signed-off-by: glorv <[email protected]>

@Connor1996 Connor1996 left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Comment thread src/raft_log.rs Outdated
/// on a quorum of nodes.
///
/// Invariant: applied <= committed
/// NOTE: this invariant can be break after restart if `max_apply_unpersisted_log_limit` is 0,

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is greater than 0?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Comment thread src/raft_log.rs
glorv added 2 commits March 28, 2024 17:24
Signed-off-by: glorv <[email protected]>
Signed-off-by: glorv <[email protected]>
Comment thread src/raft_log.rs Outdated
Comment on lines +63 to +64
/// the maximum log gap between persisted_index and applied_index.
/// Caller should ensure the value won't lead to the upper bound overflow.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment about overflow may be unnecessary. Alternatively, it can be written in Config.

Suggested change
/// 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]>
@glorv glorv force-pushed the apply-before-persist branch from f66d2bb to 47c7f1d Compare March 28, 2024 10:30
@gengliqi

gengliqi commented Mar 28, 2024

Copy link
Copy Markdown
Member

I think we need to add more restrictions to these logs that can be applied before persisting so that correctness errors can be found.
My idea of the restriction is:
A log can be applied before persisting only when a log with the same term has been committed and persisted.
After restarting, we can check if the logs between committed and applied have the same term as the last committed entry. If not the same, there must be something wrong and the panic must happen immediately.

@gengliqi

Copy link
Copy Markdown
Member

I will propose another pr to implement this restriction.

@gengliqi gengliqi merged commit 3cfa667 into tikv:master Mar 28, 2024
@glorv glorv deleted the apply-before-persist branch March 28, 2024 11:22
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants