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

kv: dropping latches after failed raft reproposal is unsafe #115020

Open
nvanbenschoten opened this issue Nov 23, 2023 · 11 comments
Open

kv: dropping latches after failed raft reproposal is unsafe #115020

nvanbenschoten opened this issue Nov 23, 2023 · 11 comments
Assignees
Labels
A-kv-replication Relating to Raft, consensus, and coordination. branch-master Failures and bugs on the master branch. branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-kv KV Team

Comments

@nvanbenschoten
Copy link
Member

nvanbenschoten commented Nov 23, 2023

The following logic rejects a raft proposal with an AmbiguousResultError if an attempt to repropose it fails:

if err := r.mu.proposalBuf.ReinsertLocked(ctx, p); err != nil {
r.cleanupFailedProposalLocked(p)
p.finishApplication(ctx, makeProposalResultErr(
kvpb.NewAmbiguousResultError(err)))
continue

In doing so, it calls finishApplication, which releases latches and cleans up the request.

It's not clear how this is safe. I don't think it is. Unlike the other case where we reject requests during a raft reproposal attempt (here), on this path we have no strong reason to believe that the original proposal won't eventually succeed. If it could eventually succeed then dropping latches is unsafe, as it could allow conflicting requests to proceed and evaluate before the original request applies, only for the original request to later apply. This kind of race could lead to any number of issues, including stats inconsistencies and lost updates due to clobbered writes.

I think we want one of two things here.

One option is to ignore the error from ReinsertLocked and don't reject the proposal, allowing it to be reproposed again later. This may lead to requests like lease acquisitions getting stuck indefinitely in the proposals map, so we'd need to careful.

The other option is to signal a result to the proposal without dropping latches. This is what we (correctly) do when poisoning requests:

aErr := kvpb.NewAmbiguousResultError(err)
// NB: this does not release the request's latches. It's important that
// the latches stay in place, since the command could still apply.
p.signalProposalResult(makeProposalResultErr(aErr))

It's also possible that we never actually hit this error in practice and that the code is effectively dead. There are very few cases where ReinsertLocked returns an error. It only does when the replica is destroyed (at which point, all proposals are already rejected) and it does in rare cases when the propBuf is full and flushing it returns an error. So I might be making a big deal about a non-issue. Either way, we should fix the code to not look so error-prone.


Original Slack discussion: https://cockroachlabs.slack.com/archives/C0KB9Q03D/p1700688914732519?thread_ts=1700675982.566959&cid=C0KB9Q03D

I’ve been looking into whether raft reproposals might be involved here. I don’t have any real evidence that they are beyond that the leaseholder of the range was not the raft leader and was receiving raft snapshots, which can create more reproposal traffic.
I’ll spell out the theory:

  • n39 is the leaseholder but not the leader
  • n39 evaluates, proposes, and applies the initial version of the intent with txnDidNotUpdateMeta=true
  • n39 evaluates and proposes a (hypothetical) update (e.g. ResolveIntent(PENDING), maybe it got pushed[1]) to the intent which sets txnDidNotUpdateMeta=false. This raft proposal is committed to the log, but n39 is not the leader and is having trouble communicating with its peers, so it does not apply it immediately
  • n39 attempts to repropose this proposal, maybe because it applies a snapshot, maybe not. The reproposal fails here. An ambiguous result error is returned to the update of the intent and that request releases its latches[2]. Again, this request committed in the log, but n39 hasn’t heard that yet and hasn’t applied it locally
  • n39 evaluates the EndTxn request and sync resolves the intent. This is allowed because the previous proposal dropped its latches. The EndTxn reads txnDidNotUpdateMeta=true and proposes a write batch to raft with a SingleDel on the key.
    We’re non-deterministic from this point on. The SINGLEDEL may or may not delete both SETs on the intent key, depending on whether they clobber each other first. Replicas diverge.

[1]: The missing link here is that the committed version was at the txn’s min timestamp, which is the timestamp that it was originally written at. So there’s no indication that the intent was ever updated.
[2]: I have been able to create a stats inconsistency by randomly enabling this error path in a kvnemesis run, which gives weight to the idea that there’s something broken on this raft reproposal error-handling path.

Jira issue: CRDB-33844

Epic CRDB-37617

@nvanbenschoten nvanbenschoten added C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. A-kv-replication Relating to Raft, consensus, and coordination. T-kv-replication labels Nov 23, 2023
Copy link

blathers-crl bot commented Nov 23, 2023

cc @cockroachdb/replication

@pav-kv
Copy link
Collaborator

pav-kv commented Nov 23, 2023

Unlike the other case where we reject requests during a raft reproposal attempt (here), on this path we have no strong reason to believe that the original proposal won't eventually succeed.

We neither have a reason to believe the other case is safe. If the snapshot jumps over LAIs that were still pending, we don't have a way of telling whether the snapshot contains them as applied or skipped. The comment above the check says this much. In fact, I believe this other case is a more plausible reason of the failure we're seeing.

Agree that this code needs a clean-up either way.

@nvanbenschoten
Copy link
Member Author

We neither have a reason to believe the other case is safe.

I think we do. If the snapshot jumps over LAIs that were still pending, we don't have a way of telling whether the snapshot contains them as applied or if the proposals were skipped. However, because of the MaxLeaseIndex check, we can determine that the proposal won't ever be applied in the future. So it is safe to drop latches at this point, because the proposal is either already applied or is not and will never apply at some later point.

Copy link

blathers-crl bot commented Nov 29, 2023

Hi @erikgrinaker, please add branch-* labels to identify which branch(es) this release-blocker affects.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@erikgrinaker erikgrinaker added branch-master Failures and bugs on the master branch. branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 labels Nov 29, 2023
@erikgrinaker
Copy link
Contributor

It's unclear whether this is a pre-existing bug or was introduced with the reproposals refactor (we suspect the former), so tentatively marking this as a GA blocker until we confirm.

@nvanbenschoten
Copy link
Member Author

This same logic existed in release-23.1, so this doesn't look like a new bug:
https://github.com/cockroachdb/cockroach/blob/release-23.1/pkg/kv/kvserver/replica_raft.go#L1452-L1458

Regarding a fix, I think the solution will involve the combination of:

  1. not cleaning up proposals on errors coming from ReinsertLocked during reproposals. If an error from ReinsertLocked doesn't prove that the existing proposal won't apply, then we need to keep the proposal in r.mu.proposals.
  2. extending the if p.command.MaxLeaseIndex <= r.mu.state.LeaseAppliedIndex { condition to apply to all refresh reasons, so that we don't leak proposals that can never apply. This feels moderately risky. I'm also not sure why we don't already have such a condition. Maybe because we keep attempting to repropose under the old MaxLeaseIndex until the proposal eventually commits and either applies or gets rejected by the LAI check (and possibly reproposed with a new MaxLeaseIndex by tryReproposeWithNewLeaseIndex).

@erikgrinaker
Copy link
Contributor

@pavelkalinnikov Can you confirm @nvanbenschoten's comment above, and remove the GA-blocker label if appropriate?

@pav-kv
Copy link
Collaborator

pav-kv commented Dec 11, 2023

Yeah, the diff for the refreshProposalsLocked func between master and release-23.1 is trivial, so this aspect hasn't changed.

I'm looking at #106750 to see if there are some other changes that could cause this. In particular, there are some comments about poisoning and double calling a command done. I don't know if it's old or new.

@pav-kv
Copy link
Collaborator

pav-kv commented Dec 11, 2023

  1. not cleaning up proposals on errors coming from ReinsertLocked during reproposals. If an error from ReinsertLocked doesn't prove that the existing proposal won't apply, then we need to keep the proposal in r.mu.proposals.

Agree. Even though we might be safe here (for instance, one of the error conditions in ReinsertLocked is replica destruction - and a destroyed replica won't propose anyway), it would be nice to be extra safe.

In the replica destruction case, for example, we could delegate dropping these proposals to the destruction procedure itself.

I'm less clear about the cases when the error is for different reasons.

  1. extending the if p.command.MaxLeaseIndex <= r.mu.state.LeaseAppliedIndex { condition to apply to all refresh reasons, so that we don't leak proposals that can never apply. This feels moderately risky. I'm also not sure why we don't already have such a condition. Maybe because we keep attempting to repropose under the old MaxLeaseIndex until the proposal eventually commits and either applies or gets rejected by the LAI check (and possibly reproposed with a new MaxLeaseIndex by tryReproposeWithNewLeaseIndex).

Maybe it's safest to not close proposals from this refreshProposalsLocked function at all. I think it can all be done from the application flow. The canonical place to know that a proposal won't apply or won't be proposed again is when LAI moves (which is in the command or snapshot application flow). All the commands with a <= LAI are then candidates for status update. They just need to be scanned in a FIFO order by LAI, none will be skipped or lost.

refreshProposalsLocked can be called from tick which does not update the LAI. So it seems unnecessary to check LAIs here. Additionally, it can cause unintuitive interleavings with the proposal lifecycle.

@pav-kv
Copy link
Collaborator

pav-kv commented Dec 11, 2023

So I'm leaning towards having a clearer design, like this:

  1. Have a FIFO queue of in-flight proposals. They are added sequentially, and have consecutive "lease indices", so no fancy data structures are needed (except maybe some MPSC support for concurrent writes, like in the proposal buffer).

    • In fact, this possibly can be integrated with the proposals buffer which is a similar queue. The only difference is that we would keep proposals in the queue until they are applied or got behind LAI, not just until they are flushed to raft.
    • Integrating with the proposal buffer though can be hardened by the fact that refreshProposalsLocked wants to push duplicates to it. So we may still need 2 separate queues.
  2. When LAI state changes, wipe a prefix of this queue up to the new LAI (after reporting all the successfully applied commands). Decide on the unapplied / unknown to be applied commands:

    • For proposals that can be reproposed with a newer LAI, do it; push back to the queue correspondingly.
    • For proposals that can't be reproposed (and won't be), report a permanent failure.
    • For proposals that were jumped over (like when there was a snapshot), report an ambiguous failure and don't repropose.
  3. When a replica is destroyed, or lease moves (figure out exactly what those conditions are), wipe any outstanding proposals in the queue and prevent inserts to it. Report ambiguous errors for them.

  4. Not sure if the proposals map is needed in the first place. The above data structure is searchable by LAI in O(1) since LAI is strictly +1 incremental. The map would only be needed if we additionally need to search by the proposal ID, but I don't think we need to. We will use the command ID only for sanity checks to make sure that search in the queue by LAI finds the correct command.

  5. We do need to track multi-proposal proposals though (those that update LAI and repropose). From the above, we have the invariant that an in-flight proposal is always in the queue and always exactly once, so we can store all this information right in the queue. Or we could factor out some bits that are shared across multiple reproposals.

  6. Do not touch the proposals state (and proposals queue) from refreshProposalsLocked at all. All it needs to do is scan the queue, and insert slow proposals to the proposal buffer. This nicely separates concerns. We may replace the queue scanning with a better heap-based approach, so that we don't have to scan the entire queue all the time (as we do now).

Upd: forked design improvements to #116020.

@erikgrinaker
Copy link
Contributor

Pre-existing issue, removing the GA blocker.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-kv-replication Relating to Raft, consensus, and coordination. branch-master Failures and bugs on the master branch. branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-kv KV Team
Projects
No open projects
Status: Incoming
Development

No branches or pull requests

3 participants