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

kvserver: update TruncatedState before writing #131063

Open
wants to merge 4 commits into
base: master
Choose a base branch
from

Conversation

pav-kv
Copy link
Collaborator

@pav-kv pav-kv commented Sep 20, 2024

This PR fixes a truncation "race" that causes raft log entries being not found with ErrCompacted during normal operation.


Previously, truncations were carried out as follows:

  1. Under Replica.raftMu, write a batch to Pebble that removes a prefix of the log and updates the RaftTruncatedState in the log storage.
  2. Under Replica.{raftMu,mu}, move the in-memory RaftTruncatedState forward.

Between steps 1 and 2, there can be a goroutine holding Replica.mu that reads the log based on the previous RaftTruncatedState. It can unwillingly observe that the entries in (previous.Index, next.Index] interval are missing (the raft log returns ErrCompacted for them).

There are at least 2 Replica.mu-only RawNode.Step paths affected by this.

This PR swaps the order of the updates: the truncated state is moved forward first (along with updating the log size stats), signifying a logical deletion; and only then the entries are deleted from storage, physically. This removes the possibility of the race, and eliminates the need to handle ErrCompacted, as long as the reader respects the RaftTruncatedState when accessing the log. It makes the raft log always consistent under Replica.mu and/or Replica.raftMu.


Part of #132114, #143355
Related to #130955, #131041

@pav-kv pav-kv requested a review from a team as a code owner September 20, 2024 00:07
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@tbg
Copy link
Member

tbg commented Sep 24, 2024

I feel much more enlightened by our conversation than am by the commit message here. Below is my attempt at capturing the motivation and impact of this PR more fully, feel free to use pieces of it for an updated commit message as you deem appropriate.


Log truncations delete a prefix of the raft log. Currently, we commit the write batch that contains the truncation to pebble (under raftMu) prior to updating the in-memory metadata about the "beginning" of the log, the TruncatedState (under raftMu and replicaMu).
However, access to RawNode (the raft instance) is possible while holding only replicaMu.
We generally aim to perform no I/O under replicaMu, but as part of RACv2 (TODO issue ref) are introducing read "snapshots" that a RawNode can provide under only replicaMu and which can be read from at leisure outside of replicaMu (as long as another mechanism, in practice holding raftMu, prevents mutations of the log within the boundaries of the snapshot).
Having to consider a risk that RawNode attempts to access log entries that should still be present according to the TruncatedState but which have already been removed from the storage engine is undesirable complexity.
RawNode handles a "missing" log prefix somewhat gracefully (see ErrCompacted handling), but it is desirable to constrain the set of allowable behaviors as this significantly cuts down on the complexity of the system, which is important in light of the current RACv2 workstream and beyond.
To this end, this change updates the in-memory TruncatedState before committing the deletion, ensuring that RawNode under replicaMu will only ever assume the presence of portions of the log that aren't currently in the midst of a mutation.

Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Question about locking, but generally looks good!

@sumeerbhola sumeerbhola requested a review from tbg September 25, 2024 18:09
Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

We are going to grab LogSnapshot while holding raftMu (and holding Replica.mu) and will continue holding raftMu until we use the LogSnapshot. So it's unclear to me why this change is necessary -- if this change were straightforward, this would definitely make sense, but given errors in committing the truncation batch, it doesn't seem so.

Reviewed 5 of 7 files at r1, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @pav-kv, and @tbg)


pkg/kv/kvserver/raft_log_truncator.go line 593 at r1 (raw file):

		r.setTruncatedState(trunc.RaftTruncatedState, trunc.expectedFirstIndex, trunc.isDeltaTrusted)
		r.setTruncationDelta(trunc.logDeltaBytes)
	})

What if the batch.Commit fails? Now we think the log is truncated when it is not.


pkg/kv/kvserver/raft_log_truncator.go line 608 at r1 (raw file):

			return
		}
		r.applySideEffects(ctx, &trunc.RaftTruncatedState)

Here we are passing a *kvserverpb.RaftTruncatedState and in setTruncatedState we pass a kvserverpb.RaftTruncatedState. Why this inconsistency?
We used to always a pointer before.


pkg/kv/kvserver/replica_application_result.go line 506 at r1 (raw file):

func (r *Replica) handleTruncatedStateResult(
	ctx context.Context, t *kvserverpb.RaftTruncatedState,
) (raftLogDelta int64) {

why do we delay clearing the cache entries if we've already updated Replica.mu.state.TruncatedState?

Copy link
Collaborator Author

@pav-kv pav-kv left a comment

Choose a reason for hiding this comment

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

We are going to grab LogSnapshot while holding raftMu (and holding Replica.mu) and will continue holding raftMu until we use the LogSnapshot. So it's unclear to me why this change is necessary

"We" being RACv2 - yes. There is still a class of log storage reads done while only holding Replica.mu, 2 linked from the PR description. Since they don't lock raftMu before mu, they can load an outdated TruncatedState and observe a gap in the log.

This race is the only case that can cause ErrCompacted errors in raft codebase which is also exposed via the API etc/etc. I don't think it's worth keeping this for the sake of one race condition / quirk.

The way truncations are done is also inconsistent with how snapshots are handled (which are special kinds of truncations writing both to "log storage" and "state machine storage"). With snapshots, raft knows first, and registers an "intent" in memory not to try reading below this index (see unstable.snapshot field and unstable.maybeFirstIndex method, correspondingly). Only then the snapshot and log wipe are written/synced, and acked back to raft.

Log truncations invert this order: first the truncation is enacted, and only then TruncatedState is updated (which is an equivalent of notifying raft with an intent). When we get closer to separate-raft-log project again, this discrepancy will become more pressing, so I thought it's not worth waiting and fixed it.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)

Copy link
Collaborator Author

@pav-kv pav-kv left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @sumeerbhola, and @tbg)


pkg/kv/kvserver/raft_log_truncator.go line 593 at r1 (raw file):

Previously, sumeerbhola wrote…

What if the batch.Commit fails? Now we think the log is truncated when it is not.

In the synchronous truncations flow applies can't fail. If they do, we would panic. In the decoupled truncations flow this seems more liberal, so a valid question.

What can be a reason of a fail here? Any legitimate ones?

I think: if a log index is already planned to be truncated at, it doesn't matter if the storage write fails. Logically, the prefix is already unused. If updating TruncatedState seems risky, we should make raft aware of truncations (it's a matter of adding one int to the unstable struct), and notify raft about this intent first thing.


pkg/kv/kvserver/raft_log_truncator.go line 608 at r1 (raw file):

Previously, sumeerbhola wrote…

Here we are passing a *kvserverpb.RaftTruncatedState and in setTruncatedState we pass a kvserverpb.RaftTruncatedState. Why this inconsistency?
We used to always a pointer before.

No reason, will fix.


pkg/kv/kvserver/replica_application_result.go line 506 at r1 (raw file):

Previously, sumeerbhola wrote…

why do we delay clearing the cache entries if we've already updated Replica.mu.state.TruncatedState?

Yeah, seems movable a bit up the stack, I'll consider. Though not critical.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

There is still a class of log storage reads done while only holding Replica.mu, 2 linked from the PR description.

thanks for the pointer.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @pav-kv, and @tbg)


pkg/kv/kvserver/raft_log_truncator.go line 593 at r1 (raw file):

What can be a reason of a fail here? Any legitimate ones?

I couldn't think of one, and neither could Jackson. So adding a panic here is fine, which should unblock this PR. We just don't want a situation where we keep running without the log truncated and think it is truncated.

@tbg tbg requested review from tbg and removed request for tbg September 26, 2024 07:28
@pav-kv pav-kv force-pushed the update-truncated-state-before-writing branch from 4fe206e to 2561983 Compare March 21, 2025 17:41
@pav-kv pav-kv marked this pull request as draft March 21, 2025 17:42
@pav-kv pav-kv removed the request for review from nvanbenschoten March 21, 2025 17:42
@pav-kv
Copy link
Collaborator Author

pav-kv commented Mar 21, 2025

This PR is now placed on top the latest truncation stack clean-ups, #143271 and #143249. The last commit is the change, it's much more digestible now. Hold on reviews until the other PRs are done.

@pav-kv pav-kv force-pushed the update-truncated-state-before-writing branch 4 times, most recently from b8e0954 to 9a3d8f0 Compare March 22, 2025 00:42
Copy link
Collaborator Author

@pav-kv pav-kv left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @sumeerbhola and @tbg)


pkg/kv/kvserver/raft_log_truncator.go line 593 at r1 (raw file):

Previously, sumeerbhola wrote…

What can be a reason of a fail here? Any legitimate ones?

I couldn't think of one, and neither could Jackson. So adding a panic here is fine, which should unblock this PR. We just don't want a situation where we keep running without the log truncated and think it is truncated.

Done.


pkg/kv/kvserver/replica_application_result.go line 506 at r1 (raw file):

Previously, pav-kv (Pavel Kalinnikov) wrote…

Yeah, seems movable a bit up the stack, I'll consider. Though not critical.

This is now in the right place.

@pav-kv
Copy link
Collaborator Author

pav-kv commented Mar 22, 2025

@tbg @sumeerbhola All the pieces are now in the right places. There is some renaming / commenting to be done, but this PR is reviewable now. I also updated the PR description.

@pav-kv pav-kv marked this pull request as ready for review March 22, 2025 00:50
@pav-kv pav-kv force-pushed the update-truncated-state-before-writing branch 3 times, most recently from bff7f34 to 407e68a Compare March 22, 2025 17:43
pav-kv added 4 commits March 22, 2025 22:19
Update the Replica's in-memory TruncatedState before applying the write
batch to storage. Readers of the raft log storage who synchronize with
it via Replica.mu, and read TruncatedState, will then expect to find
entries at indices > TruncatedState.Index in the log. If we write the
batch first, and only then update TruncatedState, there is a time window
during which the log storage appears to have a gap.

Epic: none
Release note: none
Epic: none
Release note: none
Epic: none
Release note: none
Epic: none
Release note: none
@pav-kv pav-kv force-pushed the update-truncated-state-before-writing branch from 4139769 to 90dd7ee Compare March 22, 2025 22:20
@tbg tbg self-requested a review March 24, 2025 12:27
@pav-kv pav-kv requested a review from sumeerbhola March 24, 2025 12:40
Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

⚡⚡⚡


// finalizeTruncationRaftMuLocked is a post-apply handler for the raft log
Copy link
Member

Choose a reason for hiding this comment

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

stage/apply sounds reasonable for naming at first glance, but then there are questions:

  • stage doesn't actually "stage" anything that really deletes the log entries: you'd expect it to populate a batch, or something like that, to be committed in finalize. The real staging either happens in the app batch (tight) or in the truncator (loose).
  • finalize similarly only deals with sideloaded entries

I can see how this all fits into how things currently work (the truncator needs a slim handle to a Replica to update the cache etc) but it would be useful to leave a blurb on these methods that hint at the bigger picture.

It might also be helpful to untether these methods from *Replica. Instead, raftTruncatorReplica could inline these methods directly, and in replicaAppBatch we could cast the *Replica as a raftTruncatorReplica for access to these methods. That way, we don't end up with methods that are confusing when considered as methods on *Replica in isolation.

Just suggestions - take any or leave it. Once we go back to only having "one kind of truncation", it will be easier to streamline this.

Copy link
Collaborator Author

@pav-kv pav-kv Mar 24, 2025

Choose a reason for hiding this comment

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

Naming is hard :) So we have 2.5 steps, essentially:

  1. Logical truncation (called stagePendingTruncationRaftMuLocked here). Updates the in-memory RaftTruncatedState, size, etc. It begins looking like the truncation has been applied.
  2. Physical truncation (could be fully asynchronous).
    a. Writes a batch that carries out the same truncation in log storage.
    b. More physical truncation (called finalize here). Removes sideloaded files after (a) is synced.

On where to put the methods: I'd like them to be part of replicaLogStorage / logstore or something like that. Need to think more about how to consolidate things: #136109.

r.handleRaftLogDeltaResultRaftMuLockedReplicaMuLocked(pt.logDeltaBytes, isDeltaTrusted)
// Ensure the raft log size is not negative since it isn't persisted between
// server restarts.
// TODO(pav-kv): should we distrust the log size if it goes negative?
Copy link
Member

Choose a reason for hiding this comment

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

That makes sense to me at least. But (outside of the expectedFirstIndex==0 case where all bets are off) it should not be possible to arrive at a negative number through trusted updates, correct?

Copy link
Collaborator Author

@pav-kv pav-kv Mar 24, 2025

Choose a reason for hiding this comment

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

Yes, we shouldn't see a negative here, post all the cleanups. There are still a couple of possibilities:

  1. The sideloaded storage size tracking is faulty and can silently skew the log size: logstore: sideloaded storage is not atomic #136416.
  2. I don't have full confidence that the leader/leaseholder-evaluated size delta exactly matches what the local replica would have computed. Maybe not today, but I could imagine mixed-version scenarios in which raft log encodings could differ.

In some future, I would have liked if the local replica keeps track of its own size precisely and in a self-contained way. One way to achieve this is #136358 and a more general approach is here.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 4 of 14 files at r20, 5 of 7 files at r21, 1 of 1 files at r23, 2 of 2 files at r24, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @tbg)

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.

4 participants