Skip to content
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: add revised plan for loosely-coupled truncation #81113

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
52 changes: 49 additions & 3 deletions pkg/storage/replicas_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,7 @@ import (
//
// Replica deletion is sequenced as the following steps (* indicates durable
// writes):

//
// - [D1*] deletion of state machine state (iff the replica is in state
// InitializedStateMachine) and write to the RangeTombstoneKey. If prior to
// this step the range was in state InitializedStateMachine, it is now in
Expand Down Expand Up @@ -462,6 +462,52 @@ import (
// RangeAppliedState.RaftAppliedIndex to use as an additional upper bound of
// what can be truncated.
//
// Addendum: As of https://github.com/cockroachdb/cockroach/pull/80193 the
// loosely-coupled raft log truncation, described above, is disabled due to
// a performance regression in write-heavy workloads (see comment
// https://github.com/cockroachdb/cockroach/issues/78412#issuecomment-1119922463
// for conclusion of investigation). The revised plan is to
//
// - Do strongly-coupled truncation in
// CanTruncateRaftIfStateMachineIsDurable for a ReplicasStorage
// implementation that shares the same engine for the state machine and
// raft state. This relies on external code structure for correctness: the
// truncation proposal flows through raft, so we have already applied the
// state machine changes for the preceding entries. A crash will cause a
// suffix of the unsynced changes to be lost, so we
// cannot lose the state machine changes while not losing the truncation.
//
// This is the same correctness argument that the code preceding
// ReplicasStorage relies on. The separation of the RaftMutationBatch
// provided to DoRaftMutation and the MutationBatch provided to
// ApplyCommittedBatch is only more formalization of the separation that
// already exists: handleRaftReadyRaftMuLocked makes raft changes with one
// batch, and replicaAppBatch.ApplyToStateMachine is used to make changes
// to the state machine with another batch. This claim is oversimplified
// since the latter also does the raft log truncation, and raft changes
// for splits and merges, which ReplicasStorage is doing in a different
// way, but it does not change the correctness claim.
//
// - Do loosely-coupled truncation in CanTruncateRaftIfStateMachineIsDurable
// for a ReplicasStorage implementation that has different engines for the
// state machine and raft state. The RaftLogTruncator (for loosely-coupled
// truncation) will be used by ReplicasStorage for this purpose. This will
// delay log truncation, but we speculate that this will not be a
// performance problem (experiments will be needed to determine the actual
// behavior):
// - With multiple key-value pairs in a batch, the memtable for the raft
// engine will be able to store more than the corresponding state
// machine memtable where the key-value pairs get individual entries in
// the memtable. This is because of the per-entry overhead. This means
// there is a decent probability that the state machine memtable will
// start getting flushed before the corresponding raft engine memtable
// is flushed. If the flush is fast enough, we would be able to truncate
Copy link
Member

Choose a reason for hiding this comment

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

Could we also "just" run the raft instance with a larger memtable?

Copy link
Contributor

Choose a reason for hiding this comment

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

This is my thinking as well. With the separate engines, we should be able to configure it specifically for this use-case -- I'm sure there are other tweaks we could make too.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is my thinking as well. With the separate engines, we should be able to configure it specifically for this use-case -- I'm sure there are other tweaks we could make too.

// the raft log before the raft log entries are flushed.
// - The smaller raft engine will have a higher likelihood that deletes
// due to truncation get flushed to L0 while the log entry being deleted
// is also in L0. This should reduce the likelihood of wasteful
// compaction of raft log entries to lower levels.
//
// - Range merges impose an additional requirement: the merge protocol (at a
// higher layer) needs the RHS replica of a merge to have applied all raft
// entries up to a specified index and that this application is durable. To
Expand Down Expand Up @@ -497,8 +543,8 @@ import (
// InitializedStateMachine, though may have been in the middle of that state
// transition, or become inconsistent for other reasons mentioned earlier.
// That is, they are potentially in RecoveryInconsistentReplica state.
// - If RangeAppliedState.RaftAppliedIndex > HardState.Commit (or there is
// no HardState), execute the following atomically:
// - If RangeAppliedState.RaftAppliedIndex > HardState.Commit (NB: HardState
// must exist), execute the following atomically:
// - If there are no log entries or all log entries are <
// RaftAppliedIndex: remove all log entries and set
// RaftTruncatedState.{Index,Term} equal to
Expand Down