From fde6267aae08c323a1d89f5d0836cbb86f9c03fb Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 16 Nov 2022 15:27:07 +0100 Subject: [PATCH] kvserverbase: remove NonDeterministicError 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]: https://github.com/cockroachdb/cockroach/pull/91750/commits/a1d7605cc9823d38284cf447bdf02ea74249539d Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/kvserverbase/BUILD.bazel | 1 - .../kvserverbase/nondeterministic_error.go | 71 ------------------ pkg/kv/kvserver/raftlog/command.go | 3 +- .../replica_application_state_machine.go | 27 ++++--- pkg/kv/kvserver/replica_raft.go | 72 ++++++++----------- pkg/kv/kvserver/replica_test.go | 4 +- pkg/kv/kvserver/store_raft.go | 11 ++- .../lint/passes/fmtsafe/functions.go | 3 - 8 files changed, 52 insertions(+), 140 deletions(-) delete mode 100644 pkg/kv/kvserver/kvserverbase/nondeterministic_error.go diff --git a/pkg/kv/kvserver/kvserverbase/BUILD.bazel b/pkg/kv/kvserver/kvserverbase/BUILD.bazel index 59effda52634..c578710b0e06 100644 --- a/pkg/kv/kvserver/kvserverbase/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverbase/BUILD.bazel @@ -8,7 +8,6 @@ go_library( "bulk_adder.go", "forced_error.go", "knobs.go", - "nondeterministic_error.go", "raftversion.go", "stores.go", ], diff --git a/pkg/kv/kvserver/kvserverbase/nondeterministic_error.go b/pkg/kv/kvserver/kvserverbase/nondeterministic_error.go deleted file mode 100644 index 65da51d1b7bf..000000000000 --- a/pkg/kv/kvserver/kvserverbase/nondeterministic_error.go +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. -// - -package kvserverbase - -import ( - "fmt" - - "github.com/cockroachdb/errors" - "github.com/cockroachdb/redact" -) - -// NonDeterministicError is an error type that indicates that a state machine -// transition failed due to an unexpected error. Failure to perform a state -// transition is a form of non-determinism, so it can't be permitted for any -// reason during the application phase of state machine replication. The only -// acceptable recourse is to signal that the replica has become corrupted. -// -// Many errors returned by apply.Decoder and apply.StateMachine implementations -// will be instances of this type. -type NonDeterministicError struct { - wrapped error - expl redact.RedactableString -} - -// NonDeterministicErrorf creates a NonDeterministicError. -func NonDeterministicErrorf(format string, args ...interface{}) error { - err := errors.AssertionFailedWithDepthf(1, format, args...) - return &NonDeterministicError{ - wrapped: err, - expl: redact.Sprintf(format, args...), - } -} - -// NonDeterministicErrorWrapf wraps the provided error with a NonDeterministicError. -func NonDeterministicErrorWrapf(err error, format string, args ...interface{}) error { - return &NonDeterministicError{ - wrapped: errors.Wrapf(err, format, args...), - expl: redact.Sprintf(format, args...), - } -} - -// Error implements the error interface. -func (e *NonDeterministicError) Error() string { - return fmt.Sprintf("non-deterministic failure: %s", e.wrapped.Error()) -} - -// Cause implements the github.com/pkg/errors.causer interface. -func (e *NonDeterministicError) Cause() error { return e.wrapped } - -// Unwrap implements the github.com/golang/xerrors.Wrapper interface, which is -// planned to be moved to the stdlib in go 1.13. -func (e *NonDeterministicError) Unwrap() error { return e.wrapped } - -// GetRedactedNonDeterministicFailureExplanation loads message from the first wrapped -// NonDeterministicError, if any. The returned message is *redacted*, -// i.e. contains no sensitive information. -func GetRedactedNonDeterministicFailureExplanation(err error) redact.RedactableString { - if nd := (*NonDeterministicError)(nil); errors.As(err, &nd) { - return nd.expl.Redact() - } - return "???" -} diff --git a/pkg/kv/kvserver/raftlog/command.go b/pkg/kv/kvserver/raftlog/command.go index 5bedbc68f0d9..4ae40ff8e910 100644 --- a/pkg/kv/kvserver/raftlog/command.go +++ b/pkg/kv/kvserver/raftlog/command.go @@ -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" ) @@ -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 } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 2c3a014114a5..fc288fea4287 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -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", @@ -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 { @@ -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 } @@ -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. @@ -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. @@ -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( @@ -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") } } @@ -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 @@ -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 } @@ -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. diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 7da1c6bf0fb8..2749e816a882 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -12,7 +12,6 @@ package kvserver import ( "context" - "fmt" "math/rand" "sort" "strings" @@ -667,7 +666,7 @@ var noSnap IncomingSnapshot // non-sensitive cue as to what happened. func (r *Replica) handleRaftReady( ctx context.Context, inSnap IncomingSnapshot, -) (handleRaftReadyStats, string, error) { +) (handleRaftReadyStats, error) { r.raftMu.Lock() defer r.raftMu.Unlock() return r.handleRaftReadyRaftMuLocked(ctx, inSnap) @@ -680,15 +679,15 @@ func (r *Replica) handleRaftReady( // non-sensitive cue as to what happened. func (r *Replica) handleRaftReadyRaftMuLocked( ctx context.Context, inSnap IncomingSnapshot, -) (stats handleRaftReadyStats, _ string, _ error) { +) (handleRaftReadyStats, error) { // handleRaftReadyRaftMuLocked is not prepared to handle context cancellation, // so assert that it's given a non-cancellable context. if ctx.Done() != nil { - return handleRaftReadyStats{}, "", errors.AssertionFailedf( + return handleRaftReadyStats{}, errors.AssertionFailedf( "handleRaftReadyRaftMuLocked cannot be called with a cancellable context") } - stats = handleRaftReadyStats{ + stats := handleRaftReadyStats{ tBegin: timeutil.Now(), } defer func() { @@ -739,10 +738,9 @@ func (r *Replica) handleRaftReadyRaftMuLocked( r.mu.Unlock() if errors.Is(err, errRemoved) { // If we've been removed then just return. - return stats, "", nil + return stats, nil } else if err != nil { - const expl = "while checking raft group for Ready" - return stats, expl, errors.Wrap(err, expl) + return stats, errors.Wrap(err, "checking raft group for Ready") } if !hasReady { // We must update the proposal quota even if we don't have a ready. @@ -754,7 +752,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( // replica has caught up, we can release // some quota back to the pool. r.updateProposalQuotaRaftMuLocked(ctx, lastLeaderID) - return stats, "", nil + return stats, nil } logRaftReady(ctx, rd) @@ -781,8 +779,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( if !raft.IsEmptySnap(rd.Snapshot) { snapUUID, err := uuid.FromBytes(rd.Snapshot.Data) if err != nil { - const expl = "invalid snapshot id" - return stats, expl, errors.Wrap(err, expl) + return stats, errors.Wrap(err, "invalid snapshot id") } if inSnap.SnapUUID == (uuid.UUID{}) { log.Fatalf(ctx, "programming error: a snapshot application was attempted outside of the streaming snapshot codepath") @@ -801,8 +798,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( stats.tSnapBegin = timeutil.Now() if err := r.applySnapshot(ctx, inSnap, rd.Snapshot, rd.HardState, subsumedRepls); err != nil { - const expl = "while applying snapshot" - return stats, expl, errors.Wrap(err, expl) + return stats, errors.Wrap(err, "while applying snapshot") } stats.tSnapEnd = timeutil.Now() stats.snap.applied = true @@ -833,11 +829,10 @@ func (r *Replica) handleRaftReadyRaftMuLocked( // If we didn't expect Raft to have a snapshot but it has one // regardless, that is unexpected and indicates a programming // error. - err := kvserverbase.NonDeterministicErrorf( + return stats, errors.AssertionFailedf( "have inSnap=nil, but raft has a snapshot %s", raft.DescribeSnapshot(rd.Snapshot), ) - return stats, fmt.Sprint(kvserverbase.GetRedactedNonDeterministicFailureExplanation(err)), err } // If the ready struct includes entries that have been committed, these @@ -864,11 +859,11 @@ func (r *Replica) handleRaftReadyRaftMuLocked( appTask.SetMaxBatchSize(r.store.TestingKnobs().MaxApplicationBatchSize) defer appTask.Close() if err := appTask.Decode(ctx, rd.CommittedEntries); err != nil { - return stats, fmt.Sprint(kvserverbase.GetRedactedNonDeterministicFailureExplanation(err)), err + return stats, err } if knobs := r.store.TestingKnobs(); knobs == nil || !knobs.DisableCanAckBeforeApplication { if err := appTask.AckCommittedEntriesBeforeApplication(ctx, state.lastIndex); err != nil { - return stats, fmt.Sprint(kvserverbase.GetRedactedNonDeterministicFailureExplanation(err)), err + return stats, err } } @@ -946,8 +941,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( metrics: r.store.metrics, } if state, err = s.storeEntries(ctx, state, rd, &stats); err != nil { - const expl = "while storing log entries" - return stats, expl, err + return stats, errors.Wrap(err, "while storing log entries") } if len(rd.Entries) > 0 { @@ -961,8 +955,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( lastPurge := prevLastIndex // old end of the log, include in deletion purgedSize, err := maybePurgeSideloaded(ctx, r.raftMu.sideloaded, firstPurge, lastPurge, purgeTerm) if err != nil { - const expl = "while purging sideloaded storage" - return stats, expl, err + return stats, errors.Wrap(err, "while purging sideloaded storage") } state.byteSize -= purgedSize if state.byteSize < 0 { @@ -1004,14 +997,16 @@ func (r *Replica) handleRaftReadyRaftMuLocked( if len(rd.CommittedEntries) > 0 { err := appTask.ApplyCommittedEntries(ctx) stats.apply = sm.moveStats() - if errors.Is(err, apply.ErrRemoved) { - // We know that our replica has been removed. All future calls to - // r.withRaftGroup() will return errRemoved so no future Ready objects - // will be processed by this Replica. - return stats, "", err - } else if err != nil { - return stats, fmt.Sprint(kvserverbase.GetRedactedNonDeterministicFailureExplanation(err)), err + if err != nil { + // NB: this branch will be hit when the replica has been removed, + // in which case errors.Is(err, apply.ErrRemoved). Our callers + // special-case this. + // + // No future Ready objects will be processed by this Replica since + // it is now marked as destroyed. + return stats, err } + if r.store.cfg.KVAdmissionController != nil && stats.apply.followerStoreWriteBytes.NumEntries > 0 { r.store.cfg.KVAdmissionController.FollowerStoreWriteBytes( @@ -1052,7 +1047,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( // raft group we will early return before this point. This, combined with // the fact that we'll refuse to process messages intended for a higher // replica ID ensures that our replica ID could not have changed. - const expl = "during advance" r.mu.Lock() err = r.withRaftGroupLocked(true, func(raftGroup *raft.RawNode) (bool, error) { @@ -1079,7 +1073,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( r.mu.applyingEntries = false r.mu.Unlock() if err != nil { - return stats, expl, errors.Wrap(err, expl) + return stats, errors.Wrap(err, "during advance") } // NB: All early returns other than the one due to not having a ready @@ -1093,7 +1087,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( // quota back at the end of handleRaftReadyRaftMuLocked, the next write will // get blocked. r.updateProposalQuotaRaftMuLocked(ctx, lastLeaderID) - return stats, "", nil + return stats, nil } // logStore is a stub of a separated Raft log storage. @@ -1129,15 +1123,13 @@ func (s *logStore) storeEntries( // last index. thinEntries, numSideloaded, sideLoadedEntriesSize, otherEntriesSize, err := maybeSideloadEntries(ctx, rd.Entries, s.sideload) if err != nil { - const expl = "during sideloading" - return raftLogState{}, errors.Wrap(err, expl) + return raftLogState{}, errors.Wrap(err, "during sideloading") } state.byteSize += sideLoadedEntriesSize if state, err = logAppend( ctx, s.stateLoader.RaftLogPrefix(), batch, state, thinEntries, ); err != nil { - const expl = "during append" - return raftLogState{}, errors.Wrap(err, expl) + return raftLogState{}, errors.Wrap(err, "during append") } stats.appendedRegularCount += len(thinEntries) - numSideloaded stats.appendedRegularBytes += otherEntriesSize @@ -1156,8 +1148,7 @@ func (s *logStore) storeEntries( // We have both in the same batch, so there's no problem. If that ever // changes, we must write and sync the Entries before the HardState. if err := s.stateLoader.SetHardState(ctx, batch, rd.HardState); err != nil { - const expl = "during setHardState" - return raftLogState{}, errors.Wrap(err, expl) + return raftLogState{}, errors.Wrap(err, "setting HardState") } } // Synchronously commit the batch with the Raft log entries and Raft hard @@ -1177,8 +1168,7 @@ func (s *logStore) storeEntries( stats.pebbleBatchBytes = int64(batch.Len()) sync := rd.MustSync && !disableSyncRaftLog.Get(&s.settings.SV) if err := batch.Commit(sync); err != nil { - const expl = "while committing batch" - return raftLogState{}, errors.Wrap(err, expl) + return raftLogState{}, errors.Wrap(err, "committing batch") } stats.sync = sync stats.tPebbleCommitEnd = timeutil.Now() @@ -1205,14 +1195,14 @@ func splitMsgApps(msgs []raftpb.Message) (msgApps, otherMsgs []raftpb.Message) { // maybeFatalOnRaftReadyErr will fatal if err is neither nil nor // apply.ErrRemoved. -func maybeFatalOnRaftReadyErr(ctx context.Context, expl string, err error) (removed bool) { +func maybeFatalOnRaftReadyErr(ctx context.Context, err error) (removed bool) { switch { case err == nil: return false case errors.Is(err, apply.ErrRemoved): return true default: - log.FatalfDepth(ctx, 1, "%s: %+v", redact.Safe(expl), err) + log.FatalfDepth(ctx, 1, "%+v", err) panic("unreachable") } } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 1417fea8e3fa..d33b8b0db831 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -7094,11 +7094,11 @@ func TestQuotaPoolAccessOnDestroyedReplica(t *testing.T) { } }() - if _, _, err := repl.handleRaftReady(ctx, noSnap); err != nil { + if _, err := repl.handleRaftReady(ctx, noSnap); err != nil { t.Fatal(err) } - if _, _, err := repl.handleRaftReady(ctx, noSnap); err != nil { + if _, err := repl.handleRaftReady(ctx, noSnap); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 10aeab123f5d..3eccf9c7a897 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -405,7 +405,6 @@ func (s *Store) processRaftSnapshotRequest( log.Fatalf(ctx, "expected snapshot: %+v", snapHeader.RaftMessageRequest) } - var stats handleRaftReadyStats typ := removePlaceholderFailed defer func() { // In the typical case, handleRaftReadyRaftMuLocked calls through to @@ -457,10 +456,8 @@ func (s *Store) processRaftSnapshotRequest( // be there based on `stats.snap.applied` but it is a questionable use of // stats and more susceptible to bugs. typ = removePlaceholderDropped - var expl string - var err error - stats, expl, err = r.handleRaftReadyRaftMuLocked(ctx, inSnap) - maybeFatalOnRaftReadyErr(ctx, expl, err) + stats, err := r.handleRaftReadyRaftMuLocked(ctx, inSnap) + maybeFatalOnRaftReadyErr(ctx, err) if !stats.snap.applied { // This line would be hit if a snapshot was sent when it isn't necessary // (i.e. follower was able to catch up via the log in the interim) or when @@ -635,8 +632,8 @@ func (s *Store) processReady(rangeID roachpb.RangeID) { } ctx := r.raftCtx - stats, expl, err := r.handleRaftReady(ctx, noSnap) - maybeFatalOnRaftReadyErr(ctx, expl, err) + stats, err := r.handleRaftReady(ctx, noSnap) + maybeFatalOnRaftReadyErr(ctx, err) elapsed := stats.tEnd.Sub(stats.tBegin) s.metrics.RaftWorkingDurationNanos.Inc(elapsed.Nanoseconds()) s.metrics.RaftHandleReadyLatency.RecordValue(elapsed.Nanoseconds()) diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 34beeccabaae..80b6e0fe5867 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -109,9 +109,6 @@ var requireConstFmt = map[string]bool{ "(*github.com/cockroachdb/cockroach/pkg/kv/kvserver.raftLogger).Fatalf": true, "(*github.com/cockroachdb/cockroach/pkg/kv/kvserver.raftLogger).Panicf": true, - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase.NonDeterministicErrorf": true, - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase.NonDeterministicErrorWrapf": true, - "(go.etcd.io/etcd/raft/v3.Logger).Debugf": true, "(go.etcd.io/etcd/raft/v3.Logger).Infof": true, "(go.etcd.io/etcd/raft/v3.Logger).Warningf": true,