-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
storage: local raft log truncation #36262
Comments
cc. @irfansharif |
@tbg I think this would be a good project for @sumeerbhola to start on, with the eventual goal for him to help move us towards #38322, which requires local storage engine compaction state to feed back in to local log truncation decisions. I know we've talked about this issue in person, but do you mind spelling out the goals and work items of this issue? Is the eventual goal to remove the @irfansharif should also be kept in the loop here, as this has a small amount of overlap with his current work on #7807. |
Good idea. Here are my high level thoughts:
The nice thing about the strawman is that it would unify the code on follower vs leader. We'd have something like this in a single place somewhere in the raft handling loop: aboveThresh := r.mu.threshLen > logLen
newFirstIdx := r.mu.lastIndex
if aboveThresh {
if isRaftLeader {
decision := newTruncateDecision(r.raftStatusLocked())
if decision.ShouldTruncate() {
if decision.NumTruncated() > (2*r.mu.threshLen)/3 {
r.mu.threshLen = max(r.mu.threshLen, defaultThreshLen)
} else {
r.mu.threshLen = min(2*r.mu.threshLen, maxThreshLen)
}
newFirstIdx := decision.NewFirstIndex()
} else {
shouldTruncate = false
}
}
} (This is just a starting point, it might look pretty different in practice). |
@tbg basic question "put raft log queue in charge of local truncation on raft leader only (i.e. it stops proposing to raft)" -- who will truncate on followers? |
The followers need to learn how to truncate locally according to a hopefully much simpler heuristic (i.e. when the log reaches X entries, blindly truncate). I'm about to run out for my meetings, but here are a few pointers to get you started: raft_log_queue.go |
This is very incomplete with numerous TODOs representing questions that I need clarification on. Fixes cockroachdb#36262 Release note : None
Just to update this, we have migrated out of the replicated truncated state, so in principle we can do local raft log truncations now. |
This is only supported for Iterators created on the DB, and excludes data in the memtable. This will be used for cockroachdb/cockroach#36262 which is a prerequisite for separating the state machine into a different DB. Note that RocksDB supports such behavior using a ReadTier setting equal to kPersistedTier, which this PR does not adopt because it was considered too flexible (and has limitations like not supporting iterators). See https://github.com/facebook/rocksdb/blob/f6d7ec1d02de1fa84eff61b7ac5a3c663bd63cd7/include/rocksdb/options.h#L1394-L1408 https://github.com/facebook/rocksdb/blob/f6d7ec1d02de1fa84eff61b7ac5a3c663bd63cd7/include/rocksdb/options.h#L1467-L1471 Additionally, if the exclusion of memtables is an implementation decision, as outlined in the comment in IterOptions, it allows us more flexibility in implementation in the future.
In the ReplicasStorage design we stop making any assumptions regarding what is durable in the state machine when syncing a batch that commits changes to the raft log. This implies the need to make raft log truncation more loosely coupled than it is now, since we can truncate only when certain that the state machine is durable up to the truncation index. Current raft log truncation flows through raft and even though the RaftTruncatedStateKey is not a replicated key, it is coupled in the sense that the truncation is done below raft when processing the corresponding log entry (that asked for truncation to be done). The current setup also has correctness issues wrt maintaining the raft log size, when passing the delta bytes for a truncation. We compute th delta at proposal time (to avoid repeating iteration over the entries in all replicas), but we do not pass the first index corresponding to the truncation, so gaps or overlaps cannot be noticed at truncation time. We do want to contine to have the raft leader guide the truncation since we do not want either leader or followers to over-truncate, given our desire to serve snapshots from any replica. In the loosely coupled approach implemented here, the truncation request that flows through raft serves as an upper bound on what can be truncated. The truncation request includes an ExpectedFirstIndex. This is further propagated using ReplicatedEvalResult.RaftExpectedFirstIndex. This ExpectedFirstIndex allows one to notice gaps or overlaps when enacting a sequence of truncations, which results in setting the Replica.raftLogSizeTrusted to false. The correctness issue with Replica.raftLogSize is not fully addressed since there are existing consistency issues when evaluating a TruncateLogRequest (these are now noted in a code comment). Below raft, the truncation requests are queued onto a Replica in pendingLogTruncations. The queueing and dequeuing is managed by a truncatorForReplicasInStore that takes care of merging pending truncation requests and enacting the truncations when the durability of the state machine advances. The pending truncation requests are taken into account in the raftLogQueue when deciding whether to do another truncation. Most of the behavior of the raftLogQueue is unchanged. The one significant change is needed to handle the case of loss of all the pending requests when a node restarts. If the range is quiescent on restart, truncation may never happen on a follower since the leader has already replicated its guidance for truncation and truncated its own log. For this we rely on the raftLogQueue suggesting a local truncation (that does not flow through raft) for ranges that are quiescent. This local truncation truncates all entries in the raft log. The new behavior is gated on a LooselyCoupledRaftLogTruncation cluster version. Additionally, the new behavior can be turned off using the kv.raft_log.enable_loosely_coupled_truncation cluster setting, which is true by default. The latter is expected to be a safety switch for 1 release after which we expect to remove it. That removal will also cleanup some duplicated code (that was non-trivial to refactor and share) between the previous coupled and new loosely coupled truncation. Fixes cockroachdb#36262 Informs cockroachdb#16624,cockroachdb#38322 Release note: None
This is only supported for Iterators created on the DB, and excludes data in the memtable. This will be used for cockroachdb/cockroach#36262 which is a prerequisite for separating the state machine into a different DB. Note that RocksDB supports such behavior using a ReadTier setting equal to kPersistedTier, which this PR does not adopt because it was considered too flexible (and has limitations like not supporting iterators). See https://github.com/facebook/rocksdb/blob/f6d7ec1d02de1fa84eff61b7ac5a3c663bd63cd7/include/rocksdb/options.h#L1394-L1408 https://github.com/facebook/rocksdb/blob/f6d7ec1d02de1fa84eff61b7ac5a3c663bd63cd7/include/rocksdb/options.h#L1467-L1471 Additionally, if the exclusion of memtables is an implementation decision, as outlined in the comment in IterOptions, it allows us more flexibility in implementation in the future.
76012: server, sql: add VIEWCLUSTERSETTING user privilege r=koorosh a=koorosh Before, only users with `admin` role or `MODIFYCLUSTERSETTING` permission could view cluster settings. Now, new role is added to provide users view-only permission to view cluster settings from SQL shell and in Db Console (in Advanced debugging > Cluster settings). This change doesn't change behavior for `MODIFYCLUSTERSETTING` option, it also allows view and modify cluster settings. Release note (sql change): new user privileges are added: `VIEWCLUSTERSETTING` and `NOVIEWCLUSTERSETTING` that allows users to view cluster settings only. Resolves: #74692 76215: kvserver: loosely couple raft log truncation r=tbg a=sumeerbhola In the ReplicasStorage design we stop making any assumptions regarding what is durable in the state machine when syncing a batch that commits changes to the raft log. This implies the need to make raft log truncation more loosely coupled than it is now, since we can truncate only when certain that the state machine is durable up to the truncation index. Current raft log truncation flows through raft and even though the RaftTruncatedStateKey is not a replicated key, it is coupled in the sense that the truncation is done below raft when processing the corresponding log entry (that asked for truncation to be done). The current setup also has correctness issues wrt maintaining the raft log size, when passing the delta bytes for a truncation. We compute the delta at proposal time (to avoid repeating iteration over the entries in all replicas), but we do not pass the first index corresponding to the truncation, so gaps or overlaps cannot be noticed at truncation time. We do want to continue to have the raft leader guide the truncation since we do not want either leader or followers to over-truncate, given our desire to serve snapshots from any replica. In the loosely coupled approach implemented here, the truncation request that flows through raft serves as an upper bound on what can be truncated. The truncation request includes an ExpectedFirstIndex. This is further propagated using ReplicatedEvalResult.RaftExpectedFirstIndex. This ExpectedFirstIndex allows one to notice gaps or overlaps when enacting a sequence of truncations, which results in setting the Replica.raftLogSizeTrusted to false. The correctness issue with Replica.raftLogSize is not fully addressed since there are existing consistency issues when evaluating a TruncateLogRequest (these are now noted in a code comment). Below raft, the truncation requests are queued onto a Replica in pendingLogTruncations. The queueing and dequeuing is managed by a raftLogTruncator that takes care of merging pending truncation requests and enacting the truncations when the durability of the state machine advances. The pending truncation requests are taken into account in the raftLogQueue when deciding whether to do another truncation. Most of the behavior of the raftLogQueue is unchanged. The new behavior is gated on a LooselyCoupledRaftLogTruncation cluster version. Additionally, the new behavior can be turned off using the kv.raft_log.enable_loosely_coupled_truncation.enabled cluster setting, which is true by default. The latter is expected to be a safety switch for 1 release after which we expect to remove it. That removal will also cleanup some duplicated code (that was non-trivial to refactor and share) between the previous coupled and new loosely coupled truncation. Note, this PR is the first of two -- loosely coupled truncation is turned off via a constant in this PR. The next one will eliminate the constant and put it under the control of the cluster setting. Informs #36262 Informs #16624 Release note (ops change): The cluster setting kv.raft_log.loosely_coupled_truncation.enabled can be used to disable loosely coupled truncation. 76358: sql: support partitioned hash sharded index r=chengxiong-ruan a=chengxiong-ruan Release note (sql change): Previously, crdb blocked users from creating hash sharded index in all kinds of partitioned tables including implict partitioned tables using `PARTITION ALL BY` or `REGIONAL BY ROW`. Now we turn on the support of hash sharded index in implicit partitioned tables. Which means primary key cannot be hash sharded if a table is explicitly partitioned with `PARTITION BY` or an index cannot be hash sharded if the index is explicitly partitioned with `PARTITION BY`. Paritioning columns cannot be placed explicitly as key columns of a hash sharded index as well, including regional-by-row table's `crdb_region` column. When a hash sharded index is partitioned, ranges are pre-split within every single possible partition on shard boundaries. Each partition is split up to 16 ranges, otherwise split into the number bucket count ranges. Co-authored-by: Andrii Vorobiov <and.vorobiov@gmail.com> Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com> Co-authored-by: Chengxiong Ruan <chengxiongruan@gmail.com>
The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs cockroachdb#36262 Release note: None
The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs cockroachdb#36262 Release note: None
The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs cockroachdb#36262 Release note: None
76902: kvserver: enable loosely coupled raft log truncation r=tbg a=sumeerbhola The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs #36262 Release note: None Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com>
In the ReplicasStorage design we stop making any assumptions regarding what is durable in the state machine when syncing a batch that commits changes to the raft log. This implies the need to make raft log truncation more loosely coupled than it is now, since we can truncate only when certain that the state machine is durable up to the truncation index. Current raft log truncation flows through raft and even though the RaftTruncatedStateKey is not a replicated key, it is coupled in the sense that the truncation is done below raft when processing the corresponding log entry (that asked for truncation to be done). The current setup also has correctness issues wrt maintaining the raft log size, when passing the delta bytes for a truncation. We compute the delta at proposal time (to avoid repeating iteration over the entries in all replicas), but we do not pass the first index corresponding to the truncation, so gaps or overlaps cannot be noticed at truncation time. We do want to continue to have the raft leader guide the truncation since we do not want either leader or followers to over-truncate, given our desire to serve snapshots from any replica. In the loosely coupled approach implemented here, the truncation request that flows through raft serves as an upper bound on what can be truncated. The truncation request includes an ExpectedFirstIndex. This is further propagated using ReplicatedEvalResult.RaftExpectedFirstIndex. This ExpectedFirstIndex allows one to notice gaps or overlaps when enacting a sequence of truncations, which results in setting the Replica.raftLogSizeTrusted to false. The correctness issue with Replica.raftLogSize is not fully addressed since there are existing consistency issues when evaluating a TruncateLogRequest (these are now noted in a code comment). Below raft, the truncation requests are queued onto a Replica in pendingLogTruncations. The queueing and dequeuing is managed by a raftLogTruncator that takes care of merging pending truncation requests and enacting the truncations when the durability of the state machine advances. The pending truncation requests are taken into account in the raftLogQueue when deciding whether to do another truncation. Most of the behavior of the raftLogQueue is unchanged. The new behavior is gated on a LooselyCoupledRaftLogTruncation cluster version. Additionally, the new behavior can be turned off using the kv.raft_log.enable_loosely_coupled_truncation.enabled cluster setting, which is true by default. The latter is expected to be a safety switch for 1 release after which we expect to remove it. That removal will also cleanup some duplicated code (that was non-trivial to refactor and share) between the previous coupled and new loosely coupled truncation. Note, this PR is the first of two -- loosely coupled truncation is turned off via a constant in this PR. The next one will eliminate the constant and put it under the control of the cluster setting. Informs cockroachdb#36262 Informs cockroachdb#16624,cockroachdb#38322 Release note (ops change): The cluster setting kv.raft_log.loosely_coupled_truncation.enabled can be used to disable loosely coupled truncation.
The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs cockroachdb#36262 Release note: None
In the ReplicasStorage design we stop making any assumptions regarding what is durable in the state machine when syncing a batch that commits changes to the raft log. This implies the need to make raft log truncation more loosely coupled than it is now, since we can truncate only when certain that the state machine is durable up to the truncation index. Current raft log truncation flows through raft and even though the RaftTruncatedStateKey is not a replicated key, it is coupled in the sense that the truncation is done below raft when processing the corresponding log entry (that asked for truncation to be done). The current setup also has correctness issues wrt maintaining the raft log size, when passing the delta bytes for a truncation. We compute the delta at proposal time (to avoid repeating iteration over the entries in all replicas), but we do not pass the first index corresponding to the truncation, so gaps or overlaps cannot be noticed at truncation time. We do want to continue to have the raft leader guide the truncation since we do not want either leader or followers to over-truncate, given our desire to serve snapshots from any replica. In the loosely coupled approach implemented here, the truncation request that flows through raft serves as an upper bound on what can be truncated. The truncation request includes an ExpectedFirstIndex. This is further propagated using ReplicatedEvalResult.RaftExpectedFirstIndex. This ExpectedFirstIndex allows one to notice gaps or overlaps when enacting a sequence of truncations, which results in setting the Replica.raftLogSizeTrusted to false. The correctness issue with Replica.raftLogSize is not fully addressed since there are existing consistency issues when evaluating a TruncateLogRequest (these are now noted in a code comment). Below raft, the truncation requests are queued onto a Replica in pendingLogTruncations. The queueing and dequeuing is managed by a raftLogTruncator that takes care of merging pending truncation requests and enacting the truncations when the durability of the state machine advances. The pending truncation requests are taken into account in the raftLogQueue when deciding whether to do another truncation. Most of the behavior of the raftLogQueue is unchanged. The new behavior is gated on a LooselyCoupledRaftLogTruncation cluster version. Additionally, the new behavior can be turned off using the kv.raft_log.enable_loosely_coupled_truncation.enabled cluster setting, which is true by default. The latter is expected to be a safety switch for 1 release after which we expect to remove it. That removal will also cleanup some duplicated code (that was non-trivial to refactor and share) between the previous coupled and new loosely coupled truncation. Note, this PR is the first of two -- loosely coupled truncation is turned off via a constant in this PR. The next one will eliminate the constant and put it under the control of the cluster setting. Informs cockroachdb#36262 Informs cockroachdb#16624,cockroachdb#38322 Release note (ops change): The cluster setting kv.raft_log.loosely_coupled_truncation.enabled can be used to disable loosely coupled truncation.
The const override that disabled loosely coupled truncation is removed. There is additional testing of both the end-to-end path and the truncator. Informs cockroachdb#36262 Release note: None
We now have loosely-coupled raft log truncation, where the raft leader continues to propose the truncation, and the followers queue up these truncations as pending truncations, and apply them when the guaranteed-durable state in their state machine is past the truncation point. The
|
loosely-coupled raft log truncation was disabled due to the performance regression explained in #78412 (comment) |
Next steps: #78412 (comment) Basically this issue is blocked on having two actual separate engines. As is, with logs and state sharing an engine, when the state flushes we also flushed the log, so the truncation will come too late. So in effect we do this issue when we actually do two engines (we can't keep using strongly coupled truncations with two engines) |
We can now truncate the Raft logs locally without going through quorum (since we unreplicated the TruncatedState). We can use this to make performance-optimized truncation decisions. For example, on followers we may chose to not keep any Raft log, while on leaders we may want to simplify the heuristics used by the Raft log queue today and make them local decisions, too.
Jira issue: CRDB-4513
Epic CRDB-39898
The text was updated successfully, but these errors were encountered: