Skip to content

Commit

Permalink
kv: combine local and global uncertainty limit into Interval struct
Browse files Browse the repository at this point in the history
This commit combines the server-side "local" and "global" uncertainty
limits into an `uncertainty.Interval` struct. It also centralizes the
computation of this interval to an `uncertainty.ComputeInterval` function.

As a result, MVCC no longer reaches inside of a `Transaction` object to
construct its implied uncertainty interval. Instead, an uncertainty
interval is supplied to MVCC directly.

This refactor is made in preparation for a follow-on commit that will
address cockroachdb#58459 (giving non-transactional requests uncertainty intervals),
which in turn will prepare to simplify HLC handling throughout the system.

The refactor also cleanly addresses a goal I've had to make the local
uncertainty limit a `ClockTimestamp`, which helps clarify its role in
the system.
  • Loading branch information
nvanbenschoten committed Nov 28, 2021
1 parent 534e451 commit 209e4ec
Show file tree
Hide file tree
Showing 26 changed files with 317 additions and 283 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,7 @@ go_test(
"//pkg/kv/kvserver/tenantrate",
"//pkg/kv/kvserver/tscache",
"//pkg/kv/kvserver/txnwait",
"//pkg/kv/kvserver/uncertainty",
"//pkg/roachpb:with-mocks",
"//pkg/rpc",
"//pkg/rpc/nodedialer",
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ go_library(
"//pkg/kv/kvserver/spanset",
"//pkg/kv/kvserver/stateloader",
"//pkg/kv/kvserver/txnwait",
"//pkg/kv/kvserver/uncertainty",
"//pkg/roachpb:with-mocks",
"//pkg/security",
"//pkg/settings",
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/batcheval/cmd_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,11 @@ func Get(
var intent *roachpb.Intent
var err error
val, intent, err = storage.MVCCGet(ctx, reader, args.Key, h.Timestamp, storage.MVCCGetOptions{
Inconsistent: h.ReadConsistency != roachpb.CONSISTENT,
Txn: h.Txn,
FailOnMoreRecent: args.KeyLocking != lock.None,
LocalUncertaintyLimit: cArgs.LocalUncertaintyLimit,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
Inconsistent: h.ReadConsistency != roachpb.CONSISTENT,
Txn: h.Txn,
FailOnMoreRecent: args.KeyLocking != lock.None,
Uncertainty: cArgs.Uncertainty,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
})
if err != nil {
return result.Result{}, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func Scan(
opts := storage.MVCCScanOptions{
Inconsistent: h.ReadConsistency != roachpb.CONSISTENT,
Txn: h.Txn,
LocalUncertaintyLimit: cArgs.LocalUncertaintyLimit,
Uncertainty: cArgs.Uncertainty,
MaxKeys: h.MaxSpanRequestKeys,
MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV),
TargetBytes: h.TargetBytes,
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/declare.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
)

// DefaultDeclareKeys is the default implementation of Command.DeclareKeys.
Expand Down Expand Up @@ -106,6 +106,6 @@ type CommandArgs struct {
Header roachpb.Header
Args roachpb.Request
// *Stats should be mutated to reflect any writes made by the command.
Stats *enginepb.MVCCStats
LocalUncertaintyLimit hlc.Timestamp
Stats *enginepb.MVCCStats
Uncertainty uncertainty.Interval
}
17 changes: 9 additions & 8 deletions pkg/kv/kvserver/replica_evaluate.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -147,7 +148,7 @@ func evaluateBatch(
rec batcheval.EvalContext,
ms *enginepb.MVCCStats,
ba *roachpb.BatchRequest,
lul hlc.Timestamp,
ui uncertainty.Interval,
readOnly bool,
) (_ *roachpb.BatchResponse, _ result.Result, retErr *roachpb.Error) {

Expand Down Expand Up @@ -266,7 +267,7 @@ func evaluateBatch(
// may carry a response transaction and in the case of WriteTooOldError
// (which is sometimes deferred) it is fully populated.
curResult, err := evaluateCommand(
ctx, readWriter, rec, ms, baHeader, args, reply, lul)
ctx, readWriter, rec, ms, baHeader, args, reply, ui)

if filter := rec.EvalKnobs().TestingPostEvalFilter; filter != nil {
filterArgs := kvserverbase.FilterArgs{
Expand Down Expand Up @@ -473,18 +474,18 @@ func evaluateCommand(
h roachpb.Header,
args roachpb.Request,
reply roachpb.Response,
lul hlc.Timestamp,
ui uncertainty.Interval,
) (result.Result, error) {
var err error
var pd result.Result

if cmd, ok := batcheval.LookupCommand(args.Method()); ok {
cArgs := batcheval.CommandArgs{
EvalCtx: rec,
Header: h,
Args: args,
Stats: ms,
LocalUncertaintyLimit: lul,
EvalCtx: rec,
Header: h,
Args: args,
Stats: ms,
Uncertainty: ui,
}

if cmd.EvalRW != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/replica_evaluate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
Expand Down Expand Up @@ -664,7 +665,7 @@ func TestEvaluateBatch(t *testing.T) {
d.MockEvalCtx.EvalContext(),
&d.ms,
&d.ba,
hlc.Timestamp{},
uncertainty.Interval{},
d.readOnly,
)

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/replica_gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -169,7 +169,7 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked(
defer rw.Close()

br, result, pErr :=
evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, hlc.Timestamp{} /* lul */, true /* readOnly */)
evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, uncertainty.Interval{}, true /* readOnly */)
if pErr != nil {
return errors.Wrapf(pErr.GoError(), "couldn't scan node liveness records in span %s", span)
}
Expand Down Expand Up @@ -212,7 +212,7 @@ func (r *Replica) loadSystemConfig(ctx context.Context) (*config.SystemConfigEnt
defer rw.Close()

br, result, pErr := evaluateBatch(
ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, hlc.Timestamp{} /* lul */, true, /* readOnly */
ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, uncertainty.Interval{}, true, /* readOnly */
)
if pErr != nil {
return nil, pErr.GoError()
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
Expand Down Expand Up @@ -762,7 +762,7 @@ func (r *Replica) evaluateProposal(
ctx context.Context,
idKey kvserverbase.CmdIDKey,
ba *roachpb.BatchRequest,
lul hlc.Timestamp,
ui uncertainty.Interval,
latchSpans *spanset.SpanSet,
) (*result.Result, bool, *roachpb.Error) {
if ba.Timestamp.IsEmpty() {
Expand All @@ -779,7 +779,7 @@ func (r *Replica) evaluateProposal(
//
// TODO(tschottdorf): absorb all returned values in `res` below this point
// in the call stack as well.
batch, ms, br, res, pErr := r.evaluateWriteBatch(ctx, idKey, ba, lul, latchSpans)
batch, ms, br, res, pErr := r.evaluateWriteBatch(ctx, idKey, ba, ui, latchSpans)

// Note: reusing the proposer's batch when applying the command on the
// proposer was explored as an optimization but resulted in no performance
Expand Down Expand Up @@ -873,10 +873,10 @@ func (r *Replica) requestToProposal(
idKey kvserverbase.CmdIDKey,
ba *roachpb.BatchRequest,
st kvserverpb.LeaseStatus,
lul hlc.Timestamp,
ui uncertainty.Interval,
latchSpans *spanset.SpanSet,
) (*ProposalData, *roachpb.Error) {
res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, lul, latchSpans)
res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, ui, latchSpans)

// Fill out the results even if pErr != nil; we'll return the error below.
proposal := &ProposalData{
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util"
Expand Down Expand Up @@ -81,12 +82,12 @@ func (r *Replica) evalAndPropose(
ba *roachpb.BatchRequest,
g *concurrency.Guard,
st kvserverpb.LeaseStatus,
lul hlc.Timestamp,
ui uncertainty.Interval,
tok TrackedRequestToken,
) (chan proposalResult, func(), kvserverbase.CmdIDKey, *roachpb.Error) {
defer tok.DoneIfNotMoved(ctx)
idKey := makeIDKey()
proposal, pErr := r.requestToProposal(ctx, idKey, ba, st, lul, g.LatchSpans())
proposal, pErr := r.requestToProposal(ctx, idKey, ba, st, ui, g.LatchSpans())
log.Event(proposal.ctx, "evaluated request")

// If the request hit a server-side concurrency retry error, immediately
Expand Down
9 changes: 4 additions & 5 deletions pkg/kv/kvserver/replica_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/kr/pretty"
Expand All @@ -47,7 +46,7 @@ func (r *Replica) executeReadOnlyBatch(

// Compute the transaction's local uncertainty limit using observed
// timestamps, which can help avoid uncertainty restarts.
localUncertaintyLimit := uncertainty.ComputeLocalUncertaintyLimit(ba.Txn, st)
ui := uncertainty.ComputeInterval(ba.Txn, st)

// Evaluate read-only batch command.
spans := g.LatchSpans()
Expand Down Expand Up @@ -83,7 +82,7 @@ func (r *Replica) executeReadOnlyBatch(

var result result.Result
br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(
ctx, rw, rec, ba, localUncertaintyLimit, spans,
ctx, rw, rec, ba, ui, spans,
)

// If the request hit a server-side concurrency retry error, immediately
Expand Down Expand Up @@ -237,7 +236,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes(
rw storage.ReadWriter,
rec batcheval.EvalContext,
ba *roachpb.BatchRequest,
lul hlc.Timestamp,
ui uncertainty.Interval,
latchSpans *spanset.SpanSet,
) (br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) {
log.Event(ctx, "executing read-only batch")
Expand Down Expand Up @@ -288,7 +287,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes(
boundAccount.Clear(ctx)
log.VEventf(ctx, 2, "server-side retry of batch")
}
br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, lul, true /* readOnly */)
br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, ui, true /* readOnly */)
// If we can retry, set a higher batch timestamp and continue.
// Allow one retry only.
if pErr == nil || retries > 0 || !canDoServersideRetry(ctx, pErr, ba, br, latchSpans, nil /* deadline */) {
Expand Down
27 changes: 14 additions & 13 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/storage"
Expand Down Expand Up @@ -512,7 +513,7 @@ func sendLeaseRequest(r *Replica, l *roachpb.Lease) error {
ba.Add(&roachpb.RequestLeaseRequest{Lease: *l})
st := r.CurrentLeaseStatus(ctx)
_, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, _, pErr := r.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, pErr := r.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr == nil {
// Next if the command was committed, wait for the range to apply it.
// TODO(bdarnell): refactor this to a more conventional error-handling pattern.
Expand Down Expand Up @@ -1323,7 +1324,7 @@ func TestReplicaLeaseRejectUnknownRaftNodeID(t *testing.T) {
ba.Timestamp = tc.repl.store.Clock().Now()
ba.Add(&roachpb.RequestLeaseRequest{Lease: *lease})
_, tok := tc.repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, _, pErr := tc.repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, pErr := tc.repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr == nil {
// Next if the command was committed, wait for the range to apply it.
// TODO(bdarnell): refactor to a more conventional error-handling pattern.
Expand Down Expand Up @@ -7787,7 +7788,7 @@ func TestReplicaCancelRaftCommandProgress(t *testing.T) {
})
st := repl.CurrentLeaseStatus(ctx)
_, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, id, err := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, id, err := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -7857,7 +7858,7 @@ func TestReplicaBurstPendingCommandsAndRepropose(t *testing.T) {
})
_, tok := tc.repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
st := tc.repl.CurrentLeaseStatus(ctx)
ch, _, _, err := tc.repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, err := tc.repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -7971,7 +7972,7 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) {
ba.Timestamp = tc.Clock().Now()
ba.Add(&roachpb.PutRequest{RequestHeader: roachpb.RequestHeader{Key: roachpb.Key(id)}})
st := r.CurrentLeaseStatus(ctx)
cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), &ba, st, hlc.Timestamp{}, allSpans())
cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), &ba, st, uncertainty.Interval{}, allSpans())
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -8093,7 +8094,7 @@ func TestReplicaRefreshMultiple(t *testing.T) {

incCmdID = makeIDKey()
atomic.StoreInt32(&filterActive, 1)
proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, repl.CurrentLeaseStatus(ctx), hlc.Timestamp{}, allSpans())
proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, repl.CurrentLeaseStatus(ctx), uncertainty.Interval{}, allSpans())
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -8553,7 +8554,7 @@ func TestReplicaEvaluationNotTxnMutation(t *testing.T) {
assignSeqNumsForReqs(txn, &txnPut, &txnPut2)
origTxn := txn.Clone()

batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, hlc.Timestamp{}, allSpans())
batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, uncertainty.Interval{}, allSpans())
defer batch.Close()
if pErr != nil {
t.Fatal(pErr)
Expand Down Expand Up @@ -9254,7 +9255,7 @@ func TestErrorInRaftApplicationClearsIntents(t *testing.T) {
exLease, _ := repl.GetLease()
st := kvserverpb.LeaseStatus{Lease: exLease, State: kvserverpb.LeaseState_VALID}
_, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -9302,7 +9303,7 @@ func TestProposeWithAsyncConsensus(t *testing.T) {
atomic.StoreInt32(&filterActive, 1)
st := tc.repl.CurrentLeaseStatus(ctx)
_, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -9367,7 +9368,7 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) {
atomic.StoreInt32(&filterActive, 1)
st := repl.CurrentLeaseStatus(ctx)
_, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
_, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
_, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr != nil {
t.Fatal(pErr)
}
Expand All @@ -9386,7 +9387,7 @@ func TestApplyPaginatedCommittedEntries(t *testing.T) {

var pErr *roachpb.Error
_, tok := repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
ch, _, _, pErr = repl.evalAndPropose(ctx, &ba2, allSpansGuard(), st, hlc.Timestamp{}, tok.Move(ctx))
ch, _, _, pErr = repl.evalAndPropose(ctx, &ba2, allSpansGuard(), st, uncertainty.Interval{}, tok.Move(ctx))
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -12559,7 +12560,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) {
_, tok := tc.repl.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp)
sp := cfg.AmbientCtx.Tracer.StartSpan("replica send", tracing.WithForceRealSpan())
tracedCtx := tracing.ContextWithSpan(ctx, sp)
ch, _, _, pErr := tc.repl.evalAndPropose(tracedCtx, &ba, allSpansGuard(), st, hlc.Timestamp{}, tok)
ch, _, _, pErr := tc.repl.evalAndPropose(tracedCtx, &ba, allSpansGuard(), st, uncertainty.Interval{}, tok)
if pErr != nil {
t.Fatal(pErr)
}
Expand Down Expand Up @@ -12722,7 +12723,7 @@ func TestContainsEstimatesClampProposal(t *testing.T) {
ba.Timestamp = tc.Clock().Now()
req := putArgs(roachpb.Key("some-key"), []byte("some-value"))
ba.Add(&req)
proposal, err := tc.repl.requestToProposal(ctx, cmdIDKey, &ba, tc.repl.CurrentLeaseStatus(ctx), hlc.Timestamp{}, allSpans())
proposal, err := tc.repl.requestToProposal(ctx, cmdIDKey, &ba, tc.repl.CurrentLeaseStatus(ctx), uncertainty.Interval{}, allSpans())
if err != nil {
t.Error(err)
}
Expand Down
Loading

0 comments on commit 209e4ec

Please sign in to comment.