Skip to content

Commit

Permalink
kvserverbase: remove NonDeterministicError
Browse files Browse the repository at this point in the history
Reflecting on a recent change[^1] I realized we didn't need
`NonDeterministicError` any more since we have since fully embraced
`cockroachdb/errors` which provides redactable errors, so the "safe
explanation" functionality of this error implementation is unnecessary.

Furthermore, unexpected errors during raft application errors are by
default considered nondeterministic - it would be an invitation for
problems if this were otherwise. So contrary to what the name suggests,
using a `NonDeterministicError` never meant that we were doing anything
special to treat it as nondeterministic; there is really just one error
that gets special treatment, and it's in the other direction:
`apply.ErrRemoved` indicates that the replica got destroyed as a result
of command application, and is part of the regular Replica lifecycle.

Long story short, there wasn't a reason to keep `NonDeterministicError`
around, and this commit removes it.

[^1]: a1d7605

Epic: CRDB-220
Release note: None
  • Loading branch information
tbg committed Nov 16, 2022
1 parent dfe9109 commit fde6267
Show file tree
Hide file tree
Showing 8 changed files with 52 additions and 140 deletions.
1 change: 0 additions & 1 deletion pkg/kv/kvserver/kvserverbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ go_library(
"bulk_adder.go",
"forced_error.go",
"knobs.go",
"nondeterministic_error.go",
"raftversion.go",
"stores.go",
],
Expand Down
71 changes: 0 additions & 71 deletions pkg/kv/kvserver/kvserverbase/nondeterministic_error.go

This file was deleted.

3 changes: 2 additions & 1 deletion pkg/kv/kvserver/raftlog/command.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/errors"
"go.etcd.io/etcd/raft/v3/raftpb"
)

Expand Down Expand Up @@ -54,7 +55,7 @@ func (c *ReplicatedCmd) Decode(e *raftpb.Entry) error {
var err error
c.Entry, err = NewEntry(*e)
if err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "while decoding raft entry")
return errors.Wrapf(err, "while decoding raft entry")
}
return nil
}
Expand Down
27 changes: 13 additions & 14 deletions pkg/kv/kvserver/replica_application_state_machine.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,12 +216,12 @@ func (b *replicaAppBatch) Stage(
) (apply.CheckedCommand, error) {
cmd := cmdI.(*replicatedCmd)
if cmd.Index() == 0 {
return nil, kvserverbase.NonDeterministicErrorf("processRaftCommand requires a non-zero index")
return nil, errors.AssertionFailedf("processRaftCommand requires a non-zero index")
}
if idx, applied := cmd.Index(), b.state.RaftAppliedIndex; idx != applied+1 {
// If we have an out of order index, there's corruption. No sense in
// trying to update anything or running the command. Simply return.
return nil, kvserverbase.NonDeterministicErrorf("applied index jumped from %d to %d", applied, idx)
return nil, errors.AssertionFailedf("applied index jumped from %d to %d", applied, idx)
}
if log.V(4) {
log.Infof(ctx, "processing command %x: raftIndex=%d maxLeaseIndex=%d closedts=%s",
Expand Down Expand Up @@ -259,9 +259,9 @@ func (b *replicaAppBatch) Stage(
// way, it would become less of a one-off.
if splitMergeUnlock, err := b.r.maybeAcquireSplitMergeLock(ctx, cmd.Cmd); err != nil {
if cmd.Cmd.ReplicatedEvalResult.Split != nil {
err = kvserverbase.NonDeterministicErrorWrapf(err, "unable to acquire split lock")
err = errors.Wrap(err, "unable to acquire split lock")
} else {
err = kvserverbase.NonDeterministicErrorWrapf(err, "unable to acquire merge lock")
err = errors.Wrap(err, "unable to acquire merge lock")
}
return nil, err
} else if splitMergeUnlock != nil {
Expand Down Expand Up @@ -345,7 +345,7 @@ func (b *replicaAppBatch) stageWriteBatch(ctx context.Context, cmd *replicatedCm
b.mutations += mutations
}
if err := b.batch.ApplyBatchRepr(wb.Data, false); err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to apply WriteBatch")
return errors.Wrapf(err, "unable to apply WriteBatch")
}
return nil
}
Expand Down Expand Up @@ -460,7 +460,7 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch(
// An initialized replica is always contained in its descriptor.
rhsRepl, err := b.r.store.GetReplica(merge.RightDesc.RangeID)
if err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to get replica for merge")
return errors.Wrapf(err, "unable to get replica for merge")
}
// We should already have acquired the raftMu for the rhsRepl and now hold
// its unlock method in cmd.splitMergeUnlock.
Expand All @@ -487,7 +487,7 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch(
if err := rhsRepl.preDestroyRaftMuLocked(
ctx, b.batch, b.batch, mergedTombstoneReplicaID, clearRangeIDLocalOnly, mustClearRange,
); err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to destroy replica before merge")
return errors.Wrapf(err, "unable to destroy replica before merge")
}

// Shut down rangefeed processors on either side of the merge.
Expand Down Expand Up @@ -560,7 +560,7 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch(
if apply, err = handleTruncatedStateBelowRaftPreApply(
ctx, b.state.TruncatedState, res.State.TruncatedState, b.r.raftMu.stateLoader, b.batch,
); err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to handle truncated state")
return errors.Wrap(err, "unable to handle truncated state")
}
} else {
b.r.store.raftTruncator.addPendingTruncation(
Expand Down Expand Up @@ -630,7 +630,7 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch(
false, /* clearRangeIDLocalOnly */
false, /* mustUseClearRange */
); err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to destroy replica before removal")
return errors.Wrapf(err, "unable to destroy replica before removal")
}
}

Expand Down Expand Up @@ -714,7 +714,7 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error {
// See handleChangeReplicasResult().
sync := b.changeRemovesReplica
if err := b.batch.Commit(sync); err != nil {
return kvserverbase.NonDeterministicErrorWrapf(err, "unable to commit Raft entry batch")
return errors.Wrapf(err, "unable to commit Raft entry batch")
}
b.batch.Close()
b.batch = nil
Expand Down Expand Up @@ -840,15 +840,14 @@ func (b *replicaAppBatch) assertNoWriteBelowClosedTimestamp(cmd *replicatedCmd)
} else {
req.SafeString("request unknown; not leaseholder")
}
return kvserverbase.NonDeterministicErrorWrapf(errors.AssertionFailedf(
return errors.AssertionFailedf(
"command writing below closed timestamp; cmd: %x, write ts: %s, "+
"batch state closed: %s, command closed: %s, request: %s, lease: %s.\n"+
"This assertion will fire again on restart; to ignore run with env var\n"+
"COCKROACH_RAFT_CLOSEDTS_ASSERTIONS_ENABLED=false",
cmd.ID, wts,
b.state.RaftClosedTimestamp, cmd.Cmd.ClosedTimestamp,
req, b.state.Lease),
"command writing below closed timestamp")
req, b.state.Lease)
}
return nil
}
Expand Down Expand Up @@ -986,7 +985,7 @@ func (sm *replicaStateMachine) ApplySideEffects(

// On ConfChange entries, inform the raft.RawNode.
if err := sm.maybeApplyConfChange(ctx, cmd); err != nil {
return nil, kvserverbase.NonDeterministicErrorWrapf(err, "unable to apply conf change")
return nil, errors.Wrapf(err, "unable to apply conf change")
}

// Mark the command as applied and return it as an apply.AppliedCommand.
Expand Down
Loading

0 comments on commit fde6267

Please sign in to comment.